]> granicus.if.org Git - postgresql/commitdiff
Allow FOR EACH ROW triggers on partitioned tables
authorAlvaro Herrera <alvherre@alvh.no-ip.org>
Fri, 23 Mar 2018 13:48:22 +0000 (10:48 -0300)
committerAlvaro Herrera <alvherre@alvh.no-ip.org>
Fri, 23 Mar 2018 13:48:22 +0000 (10:48 -0300)
Previously, FOR EACH ROW triggers were not allowed in partitioned
tables.  Now we allow AFTER triggers on them, and on trigger creation we
cascade to create an identical trigger in each partition.  We also clone
the triggers to each partition that is created or attached later.

This means that deferred unique keys are allowed on partitioned tables,
too.

Author: Álvaro Herrera
Reviewed-by: Peter Eisentraut, Simon Riggs, Amit Langote, Robert Haas,
Thomas Munro
Discussion: https://postgr.es/m/20171229225319.ajltgss2ojkfd3kp@alvherre.pgsql

20 files changed:
doc/src/sgml/catalogs.sgml
doc/src/sgml/ref/create_trigger.sgml
src/backend/catalog/heap.c
src/backend/catalog/index.c
src/backend/catalog/pg_constraint.c
src/backend/commands/tablecmds.c
src/backend/commands/trigger.c
src/backend/commands/typecmds.c
src/backend/tcop/utility.c
src/include/catalog/catversion.h
src/include/catalog/indexing.h
src/include/catalog/pg_constraint.h
src/include/catalog/pg_constraint_fn.h
src/include/commands/trigger.h
src/test/regress/expected/oidjoins.out
src/test/regress/expected/triggers.out
src/test/regress/input/constraints.source
src/test/regress/output/constraints.source
src/test/regress/sql/oidjoins.sql
src/test/regress/sql/triggers.sql

index c030cdab4be7d6e06bde4fbdcf5f5f9877d926f8..95a5b113b98d794dcf5f0342b068a91027a5b3f8 100644 (file)
@@ -2254,6 +2254,14 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
        key, foreign key, or exclusion constraint; else 0</entry>
      </row>
 
+     <row>
+      <entry><structfield>conparentid</structfield></entry>
+      <entry><type>oid</type></entry>
+      <entry><literal><link linkend="catalog-pg-constraint"><structname>pg_constraint</structname></link>.oid</literal></entry>
+      <entry>The corresponding constraint in the parent partitioned table,
+       if this is a constraint in a partition; else 0</entry>
+     </row>
+
      <row>
       <entry><structfield>confrelid</structfield></entry>
       <entry><type>oid</type></entry>
index 784ed5b8e02c1481f92d130617394ed526a0b47d..7b971ee6b436c197f10264d03de28fe3856fb2e3 100644 (file)
@@ -512,6 +512,13 @@ UPDATE OF <replaceable>column_name1</replaceable> [, <replaceable>column_name2</
    the ones that are fired.
   </para>
 
+  <para>
+   Creating a row-level trigger on a partitioned table will cause identical
+   triggers to be created in all its existing partitions; and any partitions
+   created or attached later will contain an identical trigger, too.
+   Triggers on partitioned tables may only be <literal>AFTER</literal>.
+  </para>
+
   <para>
    Modifying a partitioned table or a table with inheritance children fires
    statement-level triggers attached to the explicitly named table, but not
index 5ed4654875eb31e6d91b00e0b186fb952f7b240c..b69bb1e2a41ae17886926a6dd333815494d6aa13 100644 (file)
@@ -2121,6 +2121,7 @@ StoreRelCheck(Relation rel, const char *ccname, Node *expr,
                                                          false,        /* Is Deferrable */
                                                          false,        /* Is Deferred */
                                                          is_validated,
+                                                         InvalidOid,   /* no parent constraint */
                                                          RelationGetRelid(rel),        /* relation */
                                                          attNos,       /* attrs in the constraint */
                                                          keycount, /* # attrs in the constraint */
index f4a1efbf549c492a99ae99a5aabe34a6b6aa4166..bfac37f9d17e29968c255e2c23232aa51574a2db 100644 (file)
@@ -1282,6 +1282,7 @@ index_constraint_create(Relation heapRelation,
                                                                   deferrable,
                                                                   initdeferred,
                                                                   true,
+                                                                  parentConstraintId,
                                                                   RelationGetRelid(heapRelation),
                                                                   indexInfo->ii_KeyAttrNumbers,
                                                                   indexInfo->ii_NumIndexAttrs,
@@ -1360,7 +1361,8 @@ index_constraint_create(Relation heapRelation,
                trigger->constrrel = NULL;
 
                (void) CreateTrigger(trigger, NULL, RelationGetRelid(heapRelation),
-                                                        InvalidOid, conOid, indexRelationId, true);
+                                                        InvalidOid, conOid, indexRelationId, InvalidOid,
+                                                        InvalidOid, NULL, true, false);
        }
 
        /*
index 731c5e4317e92445cf67e5ecd34429d8cc8b520d..4f1a27a7d343d9a123ada975a5353d5f3f094b65 100644 (file)
@@ -52,6 +52,7 @@ CreateConstraintEntry(const char *constraintName,
                                          bool isDeferrable,
                                          bool isDeferred,
                                          bool isValidated,
+                                         Oid parentConstrId,
                                          Oid relId,
                                          const int16 *constraintKey,
                                          int constraintNKeys,
@@ -170,6 +171,7 @@ CreateConstraintEntry(const char *constraintName,
        values[Anum_pg_constraint_conrelid - 1] = ObjectIdGetDatum(relId);
        values[Anum_pg_constraint_contypid - 1] = ObjectIdGetDatum(domainId);
        values[Anum_pg_constraint_conindid - 1] = ObjectIdGetDatum(indexRelId);
+       values[Anum_pg_constraint_conparentid - 1] = ObjectIdGetDatum(parentConstrId);
        values[Anum_pg_constraint_confrelid - 1] = ObjectIdGetDatum(foreignRelId);
        values[Anum_pg_constraint_confupdtype - 1] = CharGetDatum(foreignUpdateType);
        values[Anum_pg_constraint_confdeltype - 1] = CharGetDatum(foreignDeleteType);
@@ -772,6 +774,7 @@ ConstraintSetParentConstraint(Oid childConstrId, Oid parentConstrId)
        constrForm = (Form_pg_constraint) GETSTRUCT(newtup);
        constrForm->conislocal = false;
        constrForm->coninhcount++;
+       constrForm->conparentid = parentConstrId;
        CatalogTupleUpdate(constrRel, &tuple->t_self, newtup);
        ReleaseSysCache(tuple);
 
index f5c744b9f5a41f20918f3f06863dc42c40e818d4..e74fb1f46919bf31a71cc3020eb50586501f47cc 100644 (file)
@@ -487,6 +487,7 @@ static void ValidatePartitionConstraints(List **wqueue, Relation scanrel,
                                                         List *scanrel_children,
                                                         List *partConstraint,
                                                         bool validate_default);
+static void CloneRowTriggersToPartition(Relation parent, Relation partition);
 static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
 static ObjectAddress ATExecAttachPartitionIdx(List **wqueue, Relation rel,
                                                 RangeVar *name);
@@ -906,9 +907,11 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
        }
 
        /*
-        * If we're creating a partition, create now all the indexes defined in
-        * the parent.  We can't do it earlier, because DefineIndex wants to know
-        * the partition key which we just stored.
+        * If we're creating a partition, create now all the indexes and triggers
+        * defined in the parent.
+        *
+        * We can't do it earlier, because DefineIndex wants to know the partition
+        * key which we just stored.
         */
        if (stmt->partbound)
        {
@@ -949,6 +952,14 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
                }
 
                list_free(idxlist);
+
+               /*
+                * If there are any row-level triggers, clone them to the new
+                * partition.
+                */
+               if (parent->trigdesc != NULL)
+                       CloneRowTriggersToPartition(parent, rel);
+
                heap_close(parent, NoLock);
        }
 
@@ -7491,6 +7502,7 @@ ATAddForeignKeyConstraint(AlteredTableInfo *tab, Relation rel,
                                                                          fkconstraint->deferrable,
                                                                          fkconstraint->initdeferred,
                                                                          fkconstraint->initially_valid,
+                                                                         InvalidOid,   /* no parent constraint */
                                                                          RelationGetRelid(rel),
                                                                          fkattnum,
                                                                          numfks,
@@ -8445,7 +8457,7 @@ CreateFKCheckTrigger(Oid myRelOid, Oid refRelOid, Constraint *fkconstraint,
        fk_trigger->args = NIL;
 
        (void) CreateTrigger(fk_trigger, NULL, myRelOid, refRelOid, constraintOid,
-                                                indexOid, true);
+                                                indexOid, InvalidOid, InvalidOid, NULL, true, false);
 
        /* Make changes-so-far visible */
        CommandCounterIncrement();
@@ -8519,7 +8531,7 @@ createForeignKeyTriggers(Relation rel, Oid refRelOid, Constraint *fkconstraint,
        fk_trigger->args = NIL;
 
        (void) CreateTrigger(fk_trigger, NULL, refRelOid, myRelOid, constraintOid,
-                                                indexOid, true);
+                                                indexOid, InvalidOid, InvalidOid, NULL, true, false);
 
        /* Make changes-so-far visible */
        CommandCounterIncrement();
@@ -8574,7 +8586,7 @@ createForeignKeyTriggers(Relation rel, Oid refRelOid, Constraint *fkconstraint,
        fk_trigger->args = NIL;
 
        (void) CreateTrigger(fk_trigger, NULL, refRelOid, myRelOid, constraintOid,
-                                                indexOid, true);
+                                                indexOid, InvalidOid, InvalidOid, NULL, true, false);
 
        /* Make changes-so-far visible */
        CommandCounterIncrement();
@@ -11114,7 +11126,7 @@ static void
 ATExecEnableDisableTrigger(Relation rel, const char *trigname,
                                                   char fires_when, bool skip_system, LOCKMODE lockmode)
 {
-       EnableDisableTrigger(rel, trigname, fires_when, skip_system);
+       EnableDisableTrigger(rel, trigname, fires_when, skip_system, lockmode);
 }
 
 /*
@@ -14031,6 +14043,9 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
        /* Ensure there exists a correct set of indexes in the partition. */
        AttachPartitionEnsureIndexes(rel, attachrel);
 
+       /* and triggers */
+       CloneRowTriggersToPartition(rel, attachrel);
+
        /*
         * Generate partition constraint from the partition bound specification.
         * If the parent itself is a partition, make sure to include its
@@ -14254,6 +14269,127 @@ AttachPartitionEnsureIndexes(Relation rel, Relation attachrel)
        MemoryContextDelete(cxt);
 }
 
+/*
+ * CloneRowTriggersToPartition
+ *             subroutine for ATExecAttachPartition/DefineRelation to create row
+ *             triggers on partitions
+ */
+static void
+CloneRowTriggersToPartition(Relation parent, Relation partition)
+{
+       Relation        pg_trigger;
+       ScanKeyData key;
+       SysScanDesc scan;
+       HeapTuple       tuple;
+       MemoryContext oldcxt,
+                               perTupCxt;
+
+       ScanKeyInit(&key, Anum_pg_trigger_tgrelid, BTEqualStrategyNumber,
+                               F_OIDEQ, ObjectIdGetDatum(RelationGetRelid(parent)));
+       pg_trigger = heap_open(TriggerRelationId, RowExclusiveLock);
+       scan = systable_beginscan(pg_trigger, TriggerRelidNameIndexId,
+                                                         true, NULL, 1, &key);
+
+       perTupCxt = AllocSetContextCreate(CurrentMemoryContext,
+                                                                         "clone trig", ALLOCSET_SMALL_SIZES);
+       oldcxt = MemoryContextSwitchTo(perTupCxt);
+
+       while (HeapTupleIsValid(tuple = systable_getnext(scan)))
+       {
+               Form_pg_trigger trigForm;
+               CreateTrigStmt *trigStmt;
+               Node       *qual = NULL;
+               Datum           value;
+               bool            isnull;
+               List       *cols = NIL;
+
+               trigForm = (Form_pg_trigger) GETSTRUCT(tuple);
+
+               /*
+                * Ignore statement-level triggers; those are not cloned.
+                */
+               if (!TRIGGER_FOR_ROW(trigForm->tgtype))
+                       continue;
+
+               /*
+                * Complain if we find an unexpected trigger type.
+                */
+               if (!TRIGGER_FOR_AFTER(trigForm->tgtype))
+                       elog(ERROR, "unexpected trigger \"%s\" found",
+                                NameStr(trigForm->tgname));
+
+               /*
+                * If there is a WHEN clause, generate a 'cooked' version of it that's
+                * appropriate for the partition.
+                */
+               value = heap_getattr(tuple, Anum_pg_trigger_tgqual,
+                                                        RelationGetDescr(pg_trigger), &isnull);
+               if (!isnull)
+               {
+                       bool            found_whole_row;
+
+                       qual = stringToNode(TextDatumGetCString(value));
+                       qual = (Node *) map_partition_varattnos((List *) qual, PRS2_OLD_VARNO,
+                                                                                                       partition, parent,
+                                                                                                       &found_whole_row);
+                       if (found_whole_row)
+                               elog(ERROR, "unexpected whole-row reference found in trigger WHEN clause");
+                       qual = (Node *) map_partition_varattnos((List *) qual, PRS2_NEW_VARNO,
+                                                                                                       partition, parent,
+                                                                                                       &found_whole_row);
+                       if (found_whole_row)
+                               elog(ERROR, "unexpected whole-row reference found in trigger WHEN clause");
+               }
+
+               /*
+                * If there is a column list, transform it to a list of column names.
+                * Note we don't need to map this list in any way ...
+                */
+               if (trigForm->tgattr.dim1 > 0)
+               {
+                       int                     i;
+
+                       for (i = 0; i < trigForm->tgattr.dim1; i++)
+                       {
+                               Form_pg_attribute col;
+
+                               col = TupleDescAttr(parent->rd_att,
+                                                                       trigForm->tgattr.values[i] - 1);
+                               cols = lappend(cols, makeString(NameStr(col->attname)));
+                       }
+               }
+
+               trigStmt = makeNode(CreateTrigStmt);
+               trigStmt->trigname = NameStr(trigForm->tgname);
+               trigStmt->relation = NULL;
+               trigStmt->funcname = NULL;      /* passed separately */
+               trigStmt->args = NULL;  /* passed separately */
+               trigStmt->row = true;
+               trigStmt->timing = trigForm->tgtype & TRIGGER_TYPE_TIMING_MASK;
+               trigStmt->events = trigForm->tgtype & TRIGGER_TYPE_EVENT_MASK;
+               trigStmt->columns = cols;
+               trigStmt->whenClause = NULL;    /* passed separately */
+               trigStmt->isconstraint = OidIsValid(trigForm->tgconstraint);
+               trigStmt->transitionRels = NIL; /* not supported at present */
+               trigStmt->deferrable = trigForm->tgdeferrable;
+               trigStmt->initdeferred = trigForm->tginitdeferred;
+               trigStmt->constrrel = NULL; /* passed separately */
+
+               CreateTrigger(trigStmt, NULL, RelationGetRelid(partition),
+                                         trigForm->tgconstrrelid, InvalidOid, InvalidOid,
+                                         trigForm->tgfoid, HeapTupleGetOid(tuple), qual,
+                                         false, true);
+
+               MemoryContextReset(perTupCxt);
+       }
+
+       MemoryContextSwitchTo(oldcxt);
+       MemoryContextDelete(perTupCxt);
+
+       systable_endscan(scan);
+       heap_close(pg_trigger, RowExclusiveLock);
+}
+
 /*
  * ALTER TABLE DETACH PARTITION
  *
index fbd176b5d03de0284c421d4e08ea9c5583bf614c..9d8df5986ecafe2cfb294991c25df30d9724be7e 100644 (file)
@@ -20,6 +20,7 @@
 #include "access/xact.h"
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
+#include "catalog/index.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaccess.h"
 #include "catalog/pg_constraint.h"
@@ -123,7 +124,20 @@ static bool before_stmt_triggers_fired(Oid relid, CmdType cmdType);
  * TRIGGER, we build a pg_constraint entry internally.)
  *
  * indexOid, if nonzero, is the OID of an index associated with the constraint.
- * We do nothing with this except store it into pg_trigger.tgconstrindid.
+ * We do nothing with this except store it into pg_trigger.tgconstrindid;
+ * but when creating a trigger for a deferrable unique constraint on a
+ * partitioned table, its children are looked up.  Note we don't cope with
+ * invalid indexes in that case.
+ *
+ * funcoid, if nonzero, is the OID of the function to invoke.  When this is
+ * given, stmt->funcname is ignored.
+ *
+ * parentTriggerOid, if nonzero, is a trigger that begets this one; so that
+ * if that trigger is dropped, this one should be too.  (This is passed as
+ * Invalid by most callers; it's set here when recursing on a partition.)
+ *
+ * If whenClause is passed, it is an already-transformed expression for
+ * WHEN.  In this case, we ignore any that may come in stmt->whenClause.
  *
  * If isInternal is true then this is an internally-generated trigger.
  * This argument sets the tgisinternal field of the pg_trigger entry, and
@@ -133,6 +147,10 @@ static bool before_stmt_triggers_fired(Oid relid, CmdType cmdType);
  * relation, as well as ACL_EXECUTE on the trigger function.  For internal
  * triggers the caller must apply any required permission checks.
  *
+ * When called on partitioned tables, this function recurses to create the
+ * trigger on all the partitions, except if isInternal is true, in which
+ * case caller is expected to execute recursion on its own.
+ *
  * Note: can return InvalidObjectAddress if we decided to not create a trigger
  * at all, but a foreign-key constraint.  This is a kluge for backwards
  * compatibility.
@@ -140,13 +158,13 @@ static bool before_stmt_triggers_fired(Oid relid, CmdType cmdType);
 ObjectAddress
 CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                          Oid relOid, Oid refRelOid, Oid constraintOid, Oid indexOid,
-                         bool isInternal)
+                         Oid funcoid, Oid parentTriggerOid, Node *whenClause,
+                         bool isInternal, bool in_partition)
 {
        int16           tgtype;
        int                     ncolumns;
        int16      *columns;
        int2vector *tgattr;
-       Node       *whenClause;
        List       *whenRtable;
        char       *qual;
        Datum           values[Natts_pg_trigger];
@@ -159,7 +177,6 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
        Relation        pgrel;
        HeapTuple       tuple;
        Oid                     fargtypes[1];   /* dummy */
-       Oid                     funcoid;
        Oid                     funcrettype;
        Oid                     trigoid;
        char            internaltrigname[NAMEDATALEN];
@@ -169,6 +186,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                                referenced;
        char       *oldtablename = NULL;
        char       *newtablename = NULL;
+       bool            partition_recurse;
 
        if (OidIsValid(relOid))
                rel = heap_open(relOid, ShareRowExclusiveLock);
@@ -179,8 +197,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
         * Triggers must be on tables or views, and there are additional
         * relation-type-specific restrictions.
         */
-       if (rel->rd_rel->relkind == RELKIND_RELATION ||
-               rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+       if (rel->rd_rel->relkind == RELKIND_RELATION)
        {
                /* Tables can't have INSTEAD OF triggers */
                if (stmt->timing != TRIGGER_TYPE_BEFORE &&
@@ -190,13 +207,53 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                                         errmsg("\"%s\" is a table",
                                                        RelationGetRelationName(rel)),
                                         errdetail("Tables cannot have INSTEAD OF triggers.")));
-               /* Disallow ROW triggers on partitioned tables */
-               if (stmt->row && rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+       }
+       else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+       {
+               /* Partitioned tables can't have INSTEAD OF triggers */
+               if (stmt->timing != TRIGGER_TYPE_BEFORE &&
+                       stmt->timing != TRIGGER_TYPE_AFTER)
                        ereport(ERROR,
                                        (errcode(ERRCODE_WRONG_OBJECT_TYPE),
-                                        errmsg("\"%s\" is a partitioned table",
+                                        errmsg("\"%s\" is a table",
                                                        RelationGetRelationName(rel)),
-                                        errdetail("Partitioned tables cannot have ROW triggers.")));
+                                        errdetail("Tables cannot have INSTEAD OF triggers.")));
+
+               /*
+                * FOR EACH ROW triggers have further restrictions
+                */
+               if (stmt->row)
+               {
+                       /*
+                        * BEFORE triggers FOR EACH ROW are forbidden, because they would
+                        * allow the user to direct the row to another partition, which
+                        * isn't implemented in the executor.
+                        */
+                       if (stmt->timing != TRIGGER_TYPE_AFTER)
+                               ereport(ERROR,
+                                               (errcode(ERRCODE_WRONG_OBJECT_TYPE),
+                                                errmsg("\"%s\" is a partitioned table",
+                                                               RelationGetRelationName(rel)),
+                                                errdetail("Partitioned tables cannot have BEFORE / FOR EACH ROW triggers.")));
+
+                       /*
+                        * Disallow use of transition tables.
+                        *
+                        * Note that we have another restriction about transition tables
+                        * in partitions; search for 'has_superclass' below for an
+                        * explanation.  The check here is just to protect from the fact
+                        * that if we allowed it here, the creation would succeed for a
+                        * partitioned table with no partitions, but would be blocked by
+                        * the other restriction when the first partition was created,
+                        * which is very unfriendly behavior.
+                        */
+                       if (stmt->transitionRels != NIL)
+                               ereport(ERROR,
+                                               (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+                                                errmsg("\"%s\" is a partitioned table",
+                                                               RelationGetRelationName(rel)),
+                                                errdetail("Triggers on partitioned tables cannot have transition tables.")));
+               }
        }
        else if (rel->rd_rel->relkind == RELKIND_VIEW)
        {
@@ -297,6 +354,18 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                }
        }
 
+       /*
+        * When called on a partitioned table to create a FOR EACH ROW trigger
+        * that's not internal, we create one trigger for each partition, too.
+        *
+        * For that, we'd better hold lock on all of them ahead of time.
+        */
+       partition_recurse = !isInternal && stmt->row &&
+               rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE;
+       if (partition_recurse)
+               list_free(find_all_inheritors(RelationGetRelid(rel),
+                                                                         ShareRowExclusiveLock, NULL));
+
        /* Compute tgtype */
        TRIGGER_CLEAR_TYPE(tgtype);
        if (stmt->row)
@@ -484,9 +553,14 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
        }
 
        /*
-        * Parse the WHEN clause, if any
+        * Parse the WHEN clause, if any and we weren't passed an already
+        * transformed one.
+        *
+        * Note that as a side effect, we fill whenRtable when parsing.  If we got
+        * an already parsed clause, this does not occur, which is what we want --
+        * no point in adding redundant dependencies below.
         */
-       if (stmt->whenClause)
+       if (!whenClause && stmt->whenClause)
        {
                ParseState *pstate;
                RangeTblEntry *rte;
@@ -577,17 +651,23 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 
                free_parsestate(pstate);
        }
-       else
+       else if (!whenClause)
        {
                whenClause = NULL;
                whenRtable = NIL;
                qual = NULL;
        }
+       else
+       {
+               qual = nodeToString(whenClause);
+               whenRtable = NIL;
+       }
 
        /*
         * Find and validate the trigger function.
         */
-       funcoid = LookupFuncName(stmt->funcname, 0, fargtypes, false);
+       if (!OidIsValid(funcoid))
+               funcoid = LookupFuncName(stmt->funcname, 0, fargtypes, false);
        if (!isInternal)
        {
                aclresult = pg_proc_aclcheck(funcoid, GetUserId(), ACL_EXECUTE);
@@ -651,6 +731,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                                                                                          stmt->deferrable,
                                                                                          stmt->initdeferred,
                                                                                          true,
+                                                                                         InvalidOid,   /* no parent */
                                                                                          RelationGetRelid(rel),
                                                                                          NULL, /* no conkey */
                                                                                          0,
@@ -733,6 +814,11 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 
        /*
         * Build the new pg_trigger tuple.
+        *
+        * When we're creating a trigger in a partition, we mark it as internal,
+        * even though we don't do the isInternal magic in this function.  This
+        * makes the triggers in partitions identical to the ones in the
+        * partitioned tables, except that they are marked internal.
         */
        memset(nulls, false, sizeof(nulls));
 
@@ -742,7 +828,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
        values[Anum_pg_trigger_tgfoid - 1] = ObjectIdGetDatum(funcoid);
        values[Anum_pg_trigger_tgtype - 1] = Int16GetDatum(tgtype);
        values[Anum_pg_trigger_tgenabled - 1] = CharGetDatum(TRIGGER_FIRES_ON_ORIGIN);
-       values[Anum_pg_trigger_tgisinternal - 1] = BoolGetDatum(isInternal);
+       values[Anum_pg_trigger_tgisinternal - 1] = BoolGetDatum(isInternal || in_partition);
        values[Anum_pg_trigger_tgconstrrelid - 1] = ObjectIdGetDatum(constrrelid);
        values[Anum_pg_trigger_tgconstrindid - 1] = ObjectIdGetDatum(indexOid);
        values[Anum_pg_trigger_tgconstraint - 1] = ObjectIdGetDatum(constraintOid);
@@ -872,9 +958,8 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                pfree(DatumGetPointer(values[Anum_pg_trigger_tgnewtable - 1]));
 
        /*
-        * Update relation's pg_class entry.  Crucial side-effect: other backends
-        * (and this one too!) are sent SI message to make them rebuild relcache
-        * entries.
+        * Update relation's pg_class entry; if necessary; and if not, send an SI
+        * message to make other backends (and this one) rebuild relcache entries.
         */
        pgrel = heap_open(RelationRelationId, RowExclusiveLock);
        tuple = SearchSysCacheCopy1(RELOID,
@@ -882,20 +967,20 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
        if (!HeapTupleIsValid(tuple))
                elog(ERROR, "cache lookup failed for relation %u",
                         RelationGetRelid(rel));
+       if (!((Form_pg_class) GETSTRUCT(tuple))->relhastriggers)
+       {
+               ((Form_pg_class) GETSTRUCT(tuple))->relhastriggers = true;
 
-       ((Form_pg_class) GETSTRUCT(tuple))->relhastriggers = true;
+               CatalogTupleUpdate(pgrel, &tuple->t_self, tuple);
 
-       CatalogTupleUpdate(pgrel, &tuple->t_self, tuple);
+               CommandCounterIncrement();
+       }
+       else
+               CacheInvalidateRelcacheByTuple(tuple);
 
        heap_freetuple(tuple);
        heap_close(pgrel, RowExclusiveLock);
 
-       /*
-        * We used to try to update the rel's relcache entry here, but that's
-        * fairly pointless since it will happen as a byproduct of the upcoming
-        * CommandCounterIncrement...
-        */
-
        /*
         * Record dependencies for trigger.  Always place a normal dependency on
         * the function.
@@ -928,11 +1013,18 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                 * User CREATE TRIGGER, so place dependencies.  We make trigger be
                 * auto-dropped if its relation is dropped or if the FK relation is
                 * dropped.  (Auto drop is compatible with our pre-7.3 behavior.)
+                *
+                * Exception: if this trigger comes from a parent partitioned table,
+                * then it's not separately drop-able, but goes away if the partition
+                * does.
                 */
                referenced.classId = RelationRelationId;
                referenced.objectId = RelationGetRelid(rel);
                referenced.objectSubId = 0;
-               recordDependencyOn(&myself, &referenced, DEPENDENCY_AUTO);
+               recordDependencyOn(&myself, &referenced, OidIsValid(parentTriggerOid) ?
+                                                  DEPENDENCY_INTERNAL_AUTO :
+                                                  DEPENDENCY_AUTO);
+
                if (OidIsValid(constrrelid))
                {
                        referenced.classId = RelationRelationId;
@@ -954,6 +1046,13 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                        referenced.objectSubId = 0;
                        recordDependencyOn(&referenced, &myself, DEPENDENCY_INTERNAL);
                }
+
+               /* Depends on the parent trigger, if there is one. */
+               if (OidIsValid(parentTriggerOid))
+               {
+                       ObjectAddressSet(referenced, TriggerRelationId, parentTriggerOid);
+                       recordDependencyOn(&myself, &referenced, DEPENDENCY_INTERNAL_AUTO);
+               }
        }
 
        /* If column-specific trigger, add normal dependencies on columns */
@@ -974,7 +1073,7 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
         * If it has a WHEN clause, add dependencies on objects mentioned in the
         * expression (eg, functions, as well as any columns used).
         */
-       if (whenClause != NULL)
+       if (whenRtable != NIL)
                recordDependencyOnExpr(&myself, whenClause, whenRtable,
                                                           DEPENDENCY_NORMAL);
 
@@ -982,6 +1081,112 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
        InvokeObjectPostCreateHookArg(TriggerRelationId, trigoid, 0,
                                                                  isInternal);
 
+       /*
+        * Lastly, create the trigger on child relations, if needed.
+        */
+       if (partition_recurse)
+       {
+               PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+               List       *idxs = NIL;
+               List       *childTbls = NIL;
+               ListCell   *l;
+               int                     i;
+               MemoryContext oldcxt,
+                                       perChildCxt;
+
+               perChildCxt = AllocSetContextCreate(CurrentMemoryContext,
+                                                                                       "part trig clone",
+                                                                                       ALLOCSET_SMALL_SIZES);
+
+               /*
+                * When a trigger is being created associated with an index, we'll
+                * need to associate the trigger in each child partition with the
+                * corresponding index on it.
+                */
+               if (OidIsValid(indexOid))
+               {
+                       ListCell   *l;
+                       List       *idxs = NIL;
+
+                       idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock);
+                       foreach(l, idxs)
+                               childTbls = lappend_oid(childTbls,
+                                                                               IndexGetRelation(lfirst_oid(l),
+                                                                                                                false));
+               }
+
+               oldcxt = MemoryContextSwitchTo(perChildCxt);
+
+               /* Iterate to create the trigger on each existing partition */
+               for (i = 0; i < partdesc->nparts; i++)
+               {
+                       Oid                     indexOnChild = InvalidOid;
+                       ListCell   *l2;
+                       CreateTrigStmt *childStmt;
+                       Relation        childTbl;
+                       Node       *qual;
+                       bool            found_whole_row;
+
+                       childTbl = heap_open(partdesc->oids[i], ShareRowExclusiveLock);
+
+                       /* Find which of the child indexes is the one on this partition */
+                       if (OidIsValid(indexOid))
+                       {
+                               forboth(l, idxs, l2, childTbls)
+                               {
+                                       if (lfirst_oid(l2) == partdesc->oids[i])
+                                       {
+                                               indexOnChild = lfirst_oid(l);
+                                               break;
+                                       }
+                               }
+                               if (!OidIsValid(indexOnChild))
+                                       elog(ERROR, "failed to find index matching index \"%s\" in partition \"%s\"",
+                                                get_rel_name(indexOid),
+                                                get_rel_name(partdesc->oids[i]));
+                       }
+
+                       /*
+                        * Initialize our fabricated parse node by copying the original
+                        * one, then resetting fields that we pass separately.
+                        */
+                       childStmt = (CreateTrigStmt *) copyObject(stmt);
+                       childStmt->funcname = NIL;
+                       childStmt->args = NIL;
+                       childStmt->whenClause = NULL;
+
+                       /* If there is a WHEN clause, create a modified copy of it */
+                       qual = copyObject(whenClause);
+                       qual = (Node *)
+                               map_partition_varattnos((List *) qual, PRS2_OLD_VARNO,
+                                                                               childTbl, rel,
+                                                                               &found_whole_row);
+                       if (found_whole_row)
+                               elog(ERROR, "unexpected whole-row reference found in trigger WHEN clause");
+                       qual = (Node *)
+                               map_partition_varattnos((List *) qual, PRS2_NEW_VARNO,
+                                                                               childTbl, rel,
+                                                                               &found_whole_row);
+                       if (found_whole_row)
+                               elog(ERROR, "unexpected whole-row reference found in trigger WHEN clause");
+
+                       CreateTrigger(childStmt, queryString,
+                                                 partdesc->oids[i], refRelOid,
+                                                 InvalidOid, indexOnChild,
+                                                 funcoid, trigoid, qual,
+                                                 isInternal, true);
+
+                       heap_close(childTbl, NoLock);
+
+                       MemoryContextReset(perChildCxt);
+               }
+
+               MemoryContextSwitchTo(oldcxt);
+               MemoryContextDelete(perChildCxt);
+               list_free(idxs);
+               list_free(childTbls);
+       }
+
        /* Keep lock on target rel until end of xact */
        heap_close(rel, NoLock);
 
@@ -1579,7 +1784,7 @@ renametrig(RenameStmt *stmt)
  */
 void
 EnableDisableTrigger(Relation rel, const char *tgname,
-                                        char fires_when, bool skip_system)
+                                        char fires_when, bool skip_system, LOCKMODE lockmode)
 {
        Relation        tgrel;
        int                     nkeys;
@@ -1642,6 +1847,27 @@ EnableDisableTrigger(Relation rel, const char *tgname,
 
                        heap_freetuple(newtup);
 
+                       /*
+                        * When altering FOR EACH ROW triggers on a partitioned table, do
+                        * the same on the partitions as well.
+                        */
+                       if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
+                               (TRIGGER_FOR_ROW(oldtrig->tgtype)))
+                       {
+                               PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+                               int                     i;
+
+                               for (i = 0; i < partdesc->nparts; i++)
+                               {
+                                       Relation        part;
+
+                                       part = relation_open(partdesc->oids[i], lockmode);
+                                       EnableDisableTrigger(part, NameStr(oldtrig->tgname),
+                                                                                fires_when, skip_system, lockmode);
+                                       heap_close(part, NoLock);       /* keep lock till commit */
+                               }
+                       }
+
                        changed = true;
                }
 
@@ -5123,6 +5349,9 @@ AfterTriggerSetState(ConstraintsSetStmt *stmt)
                 * constraints within the first search-path schema that has any
                 * matches, but disregard matches in schemas beyond the first match.
                 * (This is a bit odd but it's the historical behavior.)
+                *
+                * A constraint in a partitioned table may have corresponding
+                * constraints in the partitions.  Grab those too.
                 */
                conrel = heap_open(ConstraintRelationId, AccessShareLock);
 
@@ -5217,6 +5446,32 @@ AfterTriggerSetState(ConstraintsSetStmt *stmt)
                                                                constraint->relname)));
                }
 
+               /*
+                * Scan for any possible descendants of the constraints.  We append
+                * whatever we find to the same list that we're scanning; this has the
+                * effect that we create new scans for those, too, so if there are
+                * further descendents, we'll also catch them.
+                */
+               foreach(lc, conoidlist)
+               {
+                       Oid                     parent = lfirst_oid(lc);
+                       ScanKeyData key;
+                       SysScanDesc scan;
+                       HeapTuple       tuple;
+
+                       ScanKeyInit(&key,
+                                               Anum_pg_constraint_conparentid,
+                                               BTEqualStrategyNumber, F_OIDEQ,
+                                               ObjectIdGetDatum(parent));
+
+                       scan = systable_beginscan(conrel, ConstraintParentIndexId, true, NULL, 1, &key);
+
+                       while (HeapTupleIsValid(tuple = systable_getnext(scan)))
+                               conoidlist = lappend_oid(conoidlist, HeapTupleGetOid(tuple));
+
+                       systable_endscan(scan);
+               }
+
                heap_close(conrel, AccessShareLock);
 
                /*
index e375af4cd012b704e9f468168bd8de0e2f1bb63a..25221965e9e35219ca0effcb3fcdd54e8422a285 100644 (file)
@@ -3153,6 +3153,7 @@ domainAddConstraint(Oid domainOid, Oid domainNamespace, Oid baseTypeOid,
                                                          false,        /* Is Deferrable */
                                                          false,        /* Is Deferred */
                                                          !constr->skip_validation, /* Is Validated */
+                                                         InvalidOid,   /* no parent constraint */
                                                          InvalidOid,   /* not a relation constraint */
                                                          NULL,
                                                          0,
index ed55521a0cdd027fa24d390cbbd46dc29c530c7d..8481fcca367f34f19e6cf6e74249e4148950516c 100644 (file)
@@ -1492,7 +1492,8 @@ ProcessUtilitySlow(ParseState *pstate,
                        case T_CreateTrigStmt:
                                address = CreateTrigger((CreateTrigStmt *) parsetree,
                                                                                queryString, InvalidOid, InvalidOid,
-                                                                               InvalidOid, InvalidOid, false);
+                                                                               InvalidOid, InvalidOid, InvalidOid,
+                                                                               InvalidOid, NULL, false, false);
                                break;
 
                        case T_CreatePLangStmt:
index 0ddf6a77c1a56993f21a41ea5af88b3b843121fe..1f396a9bc7d73ad0d9edfaa5f479a99a8d88c229 100644 (file)
@@ -53,6 +53,6 @@
  */
 
 /*                                                     yyyymmddN */
-#define CATALOG_VERSION_NO     201803213
+#define CATALOG_VERSION_NO     201803231
 
 #endif
index 0bb875441e39e0e1483554fc344fcd7758308173..7dd9d108d6c7cddbe52b05b2eb452811f98b8ba2 100644 (file)
@@ -128,6 +128,8 @@ DECLARE_INDEX(pg_constraint_contypid_index, 2666, on pg_constraint using btree(c
 #define ConstraintTypidIndexId 2666
 DECLARE_UNIQUE_INDEX(pg_constraint_oid_index, 2667, on pg_constraint using btree(oid oid_ops));
 #define ConstraintOidIndexId  2667
+DECLARE_INDEX(pg_constraint_conparentid_index, 2579, on pg_constraint using btree(conparentid oid_ops));
+#define ConstraintParentIndexId        2579
 
 DECLARE_UNIQUE_INDEX(pg_conversion_default_index, 2668, on pg_conversion using btree(connamespace oid_ops, conforencoding int4_ops, contoencoding int4_ops, oid oid_ops));
 #define ConversionDefaultIndexId  2668
index 8fca86d71e7e957a4e50a732dc56186184dd9938..773713b49ddec0929e79dc7f2492c09dbba69ca2 100644 (file)
@@ -72,6 +72,12 @@ CATALOG(pg_constraint,2606)
         */
        Oid                     conindid;               /* index supporting this constraint */
 
+       /*
+        * If this constraint is on a partition inherited from a partitioned
+        * table, this is the OID of the corresponding constraint in the parent.
+        */
+       Oid                     conparentid;
+
        /*
         * These fields, plus confkey, are only meaningful for a foreign-key
         * constraint.  Otherwise confrelid is 0 and the char fields are spaces.
@@ -150,7 +156,7 @@ typedef FormData_pg_constraint *Form_pg_constraint;
  *             compiler constants for pg_constraint
  * ----------------
  */
-#define Natts_pg_constraint                                    24
+#define Natts_pg_constraint                                    25
 #define Anum_pg_constraint_conname                     1
 #define Anum_pg_constraint_connamespace                2
 #define Anum_pg_constraint_contype                     3
@@ -160,21 +166,22 @@ typedef FormData_pg_constraint *Form_pg_constraint;
 #define Anum_pg_constraint_conrelid                    7
 #define Anum_pg_constraint_contypid                    8
 #define Anum_pg_constraint_conindid                    9
-#define Anum_pg_constraint_confrelid           10
-#define Anum_pg_constraint_confupdtype         11
-#define Anum_pg_constraint_confdeltype         12
-#define Anum_pg_constraint_confmatchtype       13
-#define Anum_pg_constraint_conislocal          14
-#define Anum_pg_constraint_coninhcount         15
-#define Anum_pg_constraint_connoinherit                16
-#define Anum_pg_constraint_conkey                      17
-#define Anum_pg_constraint_confkey                     18
-#define Anum_pg_constraint_conpfeqop           19
-#define Anum_pg_constraint_conppeqop           20
-#define Anum_pg_constraint_conffeqop           21
-#define Anum_pg_constraint_conexclop           22
-#define Anum_pg_constraint_conbin                      23
-#define Anum_pg_constraint_consrc                      24
+#define Anum_pg_constraint_conparentid         10
+#define Anum_pg_constraint_confrelid           11
+#define Anum_pg_constraint_confupdtype         12
+#define Anum_pg_constraint_confdeltype         13
+#define Anum_pg_constraint_confmatchtype       14
+#define Anum_pg_constraint_conislocal          15
+#define Anum_pg_constraint_coninhcount         16
+#define Anum_pg_constraint_connoinherit                17
+#define Anum_pg_constraint_conkey                      18
+#define Anum_pg_constraint_confkey                     19
+#define Anum_pg_constraint_conpfeqop           20
+#define Anum_pg_constraint_conppeqop           21
+#define Anum_pg_constraint_conffeqop           22
+#define Anum_pg_constraint_conexclop           23
+#define Anum_pg_constraint_conbin                      24
+#define Anum_pg_constraint_consrc                      25
 
 /* ----------------
  *             initial contents of pg_constraint
index d3351f4a839e27128260c62dcac608588afde014..06a2362003ebbcf0fa8f69485d113f1826bd6767 100644 (file)
@@ -33,6 +33,7 @@ extern Oid CreateConstraintEntry(const char *constraintName,
                                          bool isDeferrable,
                                          bool isDeferred,
                                          bool isValidated,
+                                         Oid parentConstrId,
                                          Oid relId,
                                          const int16 *constraintKey,
                                          int constraintNKeys,
index ff5546cf28cc8cdd7234eb59a7a0a2b1963bf93b..a5b8610fa22a1588f0a79fc46de8ca696f73c18f 100644 (file)
@@ -159,7 +159,8 @@ extern PGDLLIMPORT int SessionReplicationRole;
 
 extern ObjectAddress CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
                          Oid relOid, Oid refRelOid, Oid constraintOid, Oid indexOid,
-                         bool isInternal);
+                         Oid funcoid, Oid parentTriggerOid, Node *whenClause,
+                         bool isInternal, bool in_partition);
 
 extern void RemoveTriggerById(Oid trigOid);
 extern Oid     get_trigger_oid(Oid relid, const char *name, bool missing_ok);
@@ -167,7 +168,7 @@ extern Oid  get_trigger_oid(Oid relid, const char *name, bool missing_ok);
 extern ObjectAddress renametrig(RenameStmt *stmt);
 
 extern void EnableDisableTrigger(Relation rel, const char *tgname,
-                                        char fires_when, bool skip_system);
+                                        char fires_when, bool skip_system, LOCKMODE lockmode);
 
 extern void RelationBuildTriggers(Relation relation);
 
index 234b44fdf29837163f307af58bdb7b8aad4e4dea..d56c70c84747c459333a4c6f7b55f3ac3885d6e9 100644 (file)
@@ -369,6 +369,14 @@ WHERE      conindid != 0 AND
 ------+----------
 (0 rows)
 
+SELECT ctid, conparentid
+FROM   pg_catalog.pg_constraint fk
+WHERE  conparentid != 0 AND
+       NOT EXISTS(SELECT 1 FROM pg_catalog.pg_constraint pk WHERE pk.oid = fk.conparentid);
+ ctid | conparentid 
+------+-------------
+(0 rows)
+
 SELECT ctid, confrelid
 FROM   pg_catalog.pg_constraint fk
 WHERE  confrelid != 0 AND
index 99be9ac6e9eb3ae6d98e2b9abf5badf60b00b2e4..53e7ae41ba7e6dbd0a68fad7341dc6d98c343432 100644 (file)
@@ -1847,7 +1847,74 @@ drop function my_trigger_function();
 drop view my_view;
 drop table my_table;
 --
--- Verify that per-statement triggers are fired for partitioned tables
+-- Verify cases that are unsupported with partitioned tables
+--
+create table parted_trig (a int) partition by list (a);
+create function trigger_nothing() returns trigger
+  language plpgsql as $$ begin end; $$;
+create trigger failed before insert or update or delete on parted_trig
+  for each row execute procedure trigger_nothing();
+ERROR:  "parted_trig" is a partitioned table
+DETAIL:  Partitioned tables cannot have BEFORE / FOR EACH ROW triggers.
+create trigger failed instead of update on parted_trig
+  for each row execute procedure trigger_nothing();
+ERROR:  "parted_trig" is a table
+DETAIL:  Tables cannot have INSTEAD OF triggers.
+create trigger failed after update on parted_trig
+  referencing old table as old_table
+  for each row execute procedure trigger_nothing();
+ERROR:  "parted_trig" is a partitioned table
+DETAIL:  Triggers on partitioned tables cannot have transition tables.
+drop table parted_trig;
+--
+-- Verify trigger creation for partitioned tables, and drop behavior
+--
+create table trigpart (a int, b int) partition by range (a);
+create table trigpart1 partition of trigpart for values from (0) to (1000);
+create trigger trg1 after insert on trigpart for each row execute procedure trigger_nothing();
+create table trigpart2 partition of trigpart for values from (1000) to (2000);
+create table trigpart3 (like trigpart);
+alter table trigpart attach partition trigpart3 for values from (2000) to (3000);
+select tgrelid::regclass, tgname, tgfoid::regproc from pg_trigger
+  where tgrelid::regclass::text like 'trigpart%' order by tgrelid::regclass::text;
+  tgrelid  | tgname |     tgfoid      
+-----------+--------+-----------------
+ trigpart  | trg1   | trigger_nothing
+ trigpart1 | trg1   | trigger_nothing
+ trigpart2 | trg1   | trigger_nothing
+ trigpart3 | trg1   | trigger_nothing
+(4 rows)
+
+drop trigger trg1 on trigpart1;        -- fail
+ERROR:  cannot drop trigger trg1 on table trigpart1 because trigger trg1 on table trigpart requires it
+HINT:  You can drop trigger trg1 on table trigpart instead.
+drop trigger trg1 on trigpart2;        -- fail
+ERROR:  cannot drop trigger trg1 on table trigpart2 because trigger trg1 on table trigpart requires it
+HINT:  You can drop trigger trg1 on table trigpart instead.
+drop trigger trg1 on trigpart3;        -- fail
+ERROR:  cannot drop trigger trg1 on table trigpart3 because trigger trg1 on table trigpart requires it
+HINT:  You can drop trigger trg1 on table trigpart instead.
+drop table trigpart2;                  -- ok, trigger should be gone in that partition
+select tgrelid::regclass, tgname, tgfoid::regproc from pg_trigger
+  where tgrelid::regclass::text like 'trigpart%' order by tgrelid::regclass::text;
+  tgrelid  | tgname |     tgfoid      
+-----------+--------+-----------------
+ trigpart  | trg1   | trigger_nothing
+ trigpart1 | trg1   | trigger_nothing
+ trigpart3 | trg1   | trigger_nothing
+(3 rows)
+
+drop trigger trg1 on trigpart;         -- ok, all gone
+select tgrelid::regclass, tgname, tgfoid::regproc from pg_trigger
+  where tgrelid::regclass::text like 'trigpart%' order by tgrelid::regclass::text;
+ tgrelid | tgname | tgfoid 
+---------+--------+--------
+(0 rows)
+
+drop table trigpart;
+drop function trigger_nothing();
+--
+-- Verify that triggers are fired for partitioned tables
 --
 create table parted_stmt_trig (a int) partition by list (a);
 create table parted_stmt_trig1 partition of parted_stmt_trig for values in (1);
@@ -1864,7 +1931,7 @@ create or replace function trigger_notice() returns trigger as $$
     return null;
   end;
   $$ language plpgsql;
--- insert/update/delete statment-level triggers on the parent
+-- insert/update/delete statement-level triggers on the parent
 create trigger trig_ins_before before insert on parted_stmt_trig
   for each statement execute procedure trigger_notice();
 create trigger trig_ins_after after insert on parted_stmt_trig
@@ -1877,36 +1944,49 @@ create trigger trig_del_before before delete on parted_stmt_trig
   for each statement execute procedure trigger_notice();
 create trigger trig_del_after after delete on parted_stmt_trig
   for each statement execute procedure trigger_notice();
+-- insert/update/delete row-level triggers on the parent
+create trigger trig_ins_after_parent after insert on parted_stmt_trig
+  for each row execute procedure trigger_notice();
+create trigger trig_upd_after_parent after update on parted_stmt_trig
+  for each row execute procedure trigger_notice();
+create trigger trig_del_after_parent after delete on parted_stmt_trig
+  for each row execute procedure trigger_notice();
 -- insert/update/delete row-level triggers on the first partition
-create trigger trig_ins_before before insert on parted_stmt_trig1
+create trigger trig_ins_before_child before insert on parted_stmt_trig1
+  for each row execute procedure trigger_notice();
+create trigger trig_ins_after_child after insert on parted_stmt_trig1
   for each row execute procedure trigger_notice();
-create trigger trig_ins_after after insert on parted_stmt_trig1
+create trigger trig_upd_before_child before update on parted_stmt_trig1
   for each row execute procedure trigger_notice();
-create trigger trig_upd_before before update on parted_stmt_trig1
+create trigger trig_upd_after_child after update on parted_stmt_trig1
   for each row execute procedure trigger_notice();
-create trigger trig_upd_after after update on parted_stmt_trig1
+create trigger trig_del_before_child before delete on parted_stmt_trig1
+  for each row execute procedure trigger_notice();
+create trigger trig_del_after_child after delete on parted_stmt_trig1
   for each row execute procedure trigger_notice();
 -- insert/update/delete statement-level triggers on the parent
-create trigger trig_ins_before before insert on parted2_stmt_trig
+create trigger trig_ins_before_3 before insert on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_ins_after after insert on parted2_stmt_trig
+create trigger trig_ins_after_3 after insert on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_upd_before before update on parted2_stmt_trig
+create trigger trig_upd_before_3 before update on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_upd_after after update on parted2_stmt_trig
+create trigger trig_upd_after_3 after update on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_del_before before delete on parted2_stmt_trig
+create trigger trig_del_before_3 before delete on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_del_after after delete on parted2_stmt_trig
+create trigger trig_del_after_3 after delete on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
 with ins (a) as (
   insert into parted2_stmt_trig values (1), (2) returning a
 ) insert into parted_stmt_trig select a from ins returning tableoid::regclass, a;
 NOTICE:  trigger trig_ins_before on parted_stmt_trig BEFORE INSERT for STATEMENT
-NOTICE:  trigger trig_ins_before on parted2_stmt_trig BEFORE INSERT for STATEMENT
-NOTICE:  trigger trig_ins_before on parted_stmt_trig1 BEFORE INSERT for ROW
-NOTICE:  trigger trig_ins_after on parted_stmt_trig1 AFTER INSERT for ROW
-NOTICE:  trigger trig_ins_after on parted2_stmt_trig AFTER INSERT for STATEMENT
+NOTICE:  trigger trig_ins_before_3 on parted2_stmt_trig BEFORE INSERT for STATEMENT
+NOTICE:  trigger trig_ins_before_child on parted_stmt_trig1 BEFORE INSERT for ROW
+NOTICE:  trigger trig_ins_after_child on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_parent on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_parent on parted_stmt_trig2 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_3 on parted2_stmt_trig AFTER INSERT for STATEMENT
 NOTICE:  trigger trig_ins_after on parted_stmt_trig AFTER INSERT for STATEMENT
      tableoid      | a 
 -------------------+---
@@ -1918,25 +1998,241 @@ with upd as (
   update parted2_stmt_trig set a = a
 ) update parted_stmt_trig  set a = a;
 NOTICE:  trigger trig_upd_before on parted_stmt_trig BEFORE UPDATE for STATEMENT
-NOTICE:  trigger trig_upd_before on parted_stmt_trig1 BEFORE UPDATE for ROW
-NOTICE:  trigger trig_upd_before on parted2_stmt_trig BEFORE UPDATE for STATEMENT
-NOTICE:  trigger trig_upd_after on parted_stmt_trig1 AFTER UPDATE for ROW
+NOTICE:  trigger trig_upd_before_child on parted_stmt_trig1 BEFORE UPDATE for ROW
+NOTICE:  trigger trig_upd_before_3 on parted2_stmt_trig BEFORE UPDATE for STATEMENT
+NOTICE:  trigger trig_upd_after_child on parted_stmt_trig1 AFTER UPDATE for ROW
+NOTICE:  trigger trig_upd_after_parent on parted_stmt_trig1 AFTER UPDATE for ROW
+NOTICE:  trigger trig_upd_after_parent on parted_stmt_trig2 AFTER UPDATE for ROW
 NOTICE:  trigger trig_upd_after on parted_stmt_trig AFTER UPDATE for STATEMENT
-NOTICE:  trigger trig_upd_after on parted2_stmt_trig AFTER UPDATE for STATEMENT
+NOTICE:  trigger trig_upd_after_3 on parted2_stmt_trig AFTER UPDATE for STATEMENT
 delete from parted_stmt_trig;
 NOTICE:  trigger trig_del_before on parted_stmt_trig BEFORE DELETE for STATEMENT
+NOTICE:  trigger trig_del_before_child on parted_stmt_trig1 BEFORE DELETE for ROW
+NOTICE:  trigger trig_del_after_parent on parted_stmt_trig2 AFTER DELETE for ROW
 NOTICE:  trigger trig_del_after on parted_stmt_trig AFTER DELETE for STATEMENT
 -- insert via copy on the parent
 copy parted_stmt_trig(a) from stdin;
 NOTICE:  trigger trig_ins_before on parted_stmt_trig BEFORE INSERT for STATEMENT
-NOTICE:  trigger trig_ins_before on parted_stmt_trig1 BEFORE INSERT for ROW
-NOTICE:  trigger trig_ins_after on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_before_child on parted_stmt_trig1 BEFORE INSERT for ROW
+NOTICE:  trigger trig_ins_after_child on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_parent on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_parent on parted_stmt_trig2 AFTER INSERT for ROW
 NOTICE:  trigger trig_ins_after on parted_stmt_trig AFTER INSERT for STATEMENT
 -- insert via copy on the first partition
 copy parted_stmt_trig1(a) from stdin;
-NOTICE:  trigger trig_ins_before on parted_stmt_trig1 BEFORE INSERT for ROW
-NOTICE:  trigger trig_ins_after on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_before_child on parted_stmt_trig1 BEFORE INSERT for ROW
+NOTICE:  trigger trig_ins_after_child on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_parent on parted_stmt_trig1 AFTER INSERT for ROW
+-- Disabling a trigger in the parent table should disable children triggers too
+alter table parted_stmt_trig disable trigger trig_ins_after_parent;
+insert into parted_stmt_trig values (1);
+NOTICE:  trigger trig_ins_before on parted_stmt_trig BEFORE INSERT for STATEMENT
+NOTICE:  trigger trig_ins_before_child on parted_stmt_trig1 BEFORE INSERT for ROW
+NOTICE:  trigger trig_ins_after_child on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after on parted_stmt_trig AFTER INSERT for STATEMENT
+alter table parted_stmt_trig enable trigger trig_ins_after_parent;
+insert into parted_stmt_trig values (1);
+NOTICE:  trigger trig_ins_before on parted_stmt_trig BEFORE INSERT for STATEMENT
+NOTICE:  trigger trig_ins_before_child on parted_stmt_trig1 BEFORE INSERT for ROW
+NOTICE:  trigger trig_ins_after_child on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after_parent on parted_stmt_trig1 AFTER INSERT for ROW
+NOTICE:  trigger trig_ins_after on parted_stmt_trig AFTER INSERT for STATEMENT
 drop table parted_stmt_trig, parted2_stmt_trig;
+-- Verify that triggers fire in alphabetical order
+create table parted_trig (a int) partition by range (a);
+create table parted_trig_1 partition of parted_trig for values from (0) to (1000)
+   partition by range (a);
+create table parted_trig_1_1 partition of parted_trig_1 for values from (0) to (100);
+create table parted_trig_2 partition of parted_trig for values from (1000) to (2000);
+create trigger zzz after insert on parted_trig for each row execute procedure trigger_notice();
+create trigger mmm after insert on parted_trig_1_1 for each row execute procedure trigger_notice();
+create trigger aaa after insert on parted_trig_1 for each row execute procedure trigger_notice();
+create trigger bbb after insert on parted_trig for each row execute procedure trigger_notice();
+create trigger qqq after insert on parted_trig_1_1 for each row execute procedure trigger_notice();
+insert into parted_trig values (50), (1500);
+NOTICE:  trigger aaa on parted_trig_1_1 AFTER INSERT for ROW
+NOTICE:  trigger bbb on parted_trig_1_1 AFTER INSERT for ROW
+NOTICE:  trigger mmm on parted_trig_1_1 AFTER INSERT for ROW
+NOTICE:  trigger qqq on parted_trig_1_1 AFTER INSERT for ROW
+NOTICE:  trigger zzz on parted_trig_1_1 AFTER INSERT for ROW
+NOTICE:  trigger bbb on parted_trig_2 AFTER INSERT for ROW
+NOTICE:  trigger zzz on parted_trig_2 AFTER INSERT for ROW
+drop table parted_trig;
+-- test irregular partitions (i.e., different column definitions),
+-- including that the WHEN clause works
+create function bark(text) returns bool language plpgsql immutable
+  as $$ begin raise notice '% <- woof!', $1; return true; end; $$;
+create or replace function trigger_notice_ab() returns trigger as $$
+  begin
+    raise notice 'trigger % on % % % for %: (a,b)=(%,%)',
+               TG_NAME, TG_TABLE_NAME, TG_WHEN, TG_OP, TG_LEVEL,
+               NEW.a, NEW.b;
+    if TG_LEVEL = 'ROW' then
+       return NEW;
+    end if;
+    return null;
+  end;
+  $$ language plpgsql;
+create table parted_irreg_ancestor (fd text, b text, fd2 int, fd3 int, a int)
+  partition by range (b);
+alter table parted_irreg_ancestor drop column fd,
+  drop column fd2, drop column fd3;
+create table parted_irreg (fd int, a int, fd2 int, b text)
+  partition by range (b);
+alter table parted_irreg drop column fd, drop column fd2;
+alter table parted_irreg_ancestor attach partition parted_irreg
+  for values from ('aaaa') to ('zzzz');
+create table parted1_irreg (b text, fd int, a int);
+alter table parted1_irreg drop column fd;
+alter table parted_irreg attach partition parted1_irreg
+  for values from ('aaaa') to ('bbbb');
+create trigger parted_trig after insert on parted_irreg
+  for each row execute procedure trigger_notice_ab();
+create trigger parted_trig_odd after insert on parted_irreg for each row
+  when (bark(new.b) AND new.a % 2 = 1) execute procedure trigger_notice_ab();
+-- we should hear barking for every insert, but parted_trig_odd only emits
+-- noise for odd values of a. parted_trig does it for all inserts.
+insert into parted_irreg values (1, 'aardvark'), (2, 'aanimals');
+NOTICE:  aardvark <- woof!
+NOTICE:  aanimals <- woof!
+NOTICE:  trigger parted_trig on parted1_irreg AFTER INSERT for ROW: (a,b)=(1,aardvark)
+NOTICE:  trigger parted_trig_odd on parted1_irreg AFTER INSERT for ROW: (a,b)=(1,aardvark)
+NOTICE:  trigger parted_trig on parted1_irreg AFTER INSERT for ROW: (a,b)=(2,aanimals)
+insert into parted1_irreg values ('aardwolf', 2);
+NOTICE:  aardwolf <- woof!
+NOTICE:  trigger parted_trig on parted1_irreg AFTER INSERT for ROW: (a,b)=(2,aardwolf)
+insert into parted_irreg_ancestor values ('aasvogel', 3);
+NOTICE:  aasvogel <- woof!
+NOTICE:  trigger parted_trig on parted1_irreg AFTER INSERT for ROW: (a,b)=(3,aasvogel)
+NOTICE:  trigger parted_trig_odd on parted1_irreg AFTER INSERT for ROW: (a,b)=(3,aasvogel)
+drop table parted_irreg_ancestor;
+--
+-- Constraint triggers and partitioned tables
+create table parted_constr_ancestor (a int, b text)
+  partition by range (b);
+create table parted_constr (a int, b text)
+  partition by range (b);
+alter table parted_constr_ancestor attach partition parted_constr
+  for values from ('aaaa') to ('zzzz');
+create table parted1_constr (a int, b text);
+alter table parted_constr attach partition parted1_constr
+  for values from ('aaaa') to ('bbbb');
+create constraint trigger parted_trig after insert on parted_constr_ancestor
+  deferrable
+  for each row execute procedure trigger_notice_ab();
+create constraint trigger parted_trig_two after insert on parted_constr
+  deferrable initially deferred
+  for each row when (bark(new.b) AND new.a % 2 = 1)
+  execute procedure trigger_notice_ab();
+-- The immediate constraint is fired immediately; the WHEN clause of the
+-- deferred constraint is also called immediately.  The deferred constraint
+-- is fired at commit time.
+begin;
+insert into parted_constr values (1, 'aardvark');
+NOTICE:  aardvark <- woof!
+NOTICE:  trigger parted_trig on parted1_constr AFTER INSERT for ROW: (a,b)=(1,aardvark)
+insert into parted1_constr values (2, 'aardwolf');
+NOTICE:  aardwolf <- woof!
+NOTICE:  trigger parted_trig on parted1_constr AFTER INSERT for ROW: (a,b)=(2,aardwolf)
+insert into parted_constr_ancestor values (3, 'aasvogel');
+NOTICE:  aasvogel <- woof!
+NOTICE:  trigger parted_trig on parted1_constr AFTER INSERT for ROW: (a,b)=(3,aasvogel)
+commit;
+NOTICE:  trigger parted_trig_two on parted1_constr AFTER INSERT for ROW: (a,b)=(1,aardvark)
+NOTICE:  trigger parted_trig_two on parted1_constr AFTER INSERT for ROW: (a,b)=(3,aasvogel)
+-- The WHEN clause is immediate, and both constraint triggers are fired at
+-- commit time.
+begin;
+set constraints parted_trig deferred;
+insert into parted_constr values (1, 'aardvark');
+NOTICE:  aardvark <- woof!
+insert into parted1_constr values (2, 'aardwolf'), (3, 'aasvogel');
+NOTICE:  aardwolf <- woof!
+NOTICE:  aasvogel <- woof!
+commit;
+NOTICE:  trigger parted_trig on parted1_constr AFTER INSERT for ROW: (a,b)=(1,aardvark)
+NOTICE:  trigger parted_trig_two on parted1_constr AFTER INSERT for ROW: (a,b)=(1,aardvark)
+NOTICE:  trigger parted_trig on parted1_constr AFTER INSERT for ROW: (a,b)=(2,aardwolf)
+NOTICE:  trigger parted_trig on parted1_constr AFTER INSERT for ROW: (a,b)=(3,aasvogel)
+NOTICE:  trigger parted_trig_two on parted1_constr AFTER INSERT for ROW: (a,b)=(3,aasvogel)
+drop table parted_constr_ancestor;
+drop function bark(text);
+-- Test that the WHEN clause is set properly to partitions
+create table parted_trigger (a int, b text) partition by range (a);
+create table parted_trigger_1 partition of parted_trigger for values from (0) to (1000);
+create table parted_trigger_2 (drp int, a int, b text);
+alter table parted_trigger_2 drop column drp;
+alter table parted_trigger attach partition parted_trigger_2 for values from (1000) to (2000);
+create trigger parted_trigger after update on parted_trigger
+  for each row when (new.a % 2 = 1 and length(old.b) >= 2) execute procedure trigger_notice_ab();
+create table parted_trigger_3 (b text, a int) partition by range (length(b));
+create table parted_trigger_3_1 partition of parted_trigger_3 for values from (1) to (3);
+create table parted_trigger_3_2 partition of parted_trigger_3 for values from (3) to (5);
+alter table parted_trigger attach partition parted_trigger_3 for values from (2000) to (3000);
+insert into parted_trigger values
+    (0, 'a'), (1, 'bbb'), (2, 'bcd'), (3, 'c'),
+       (1000, 'c'), (1001, 'ddd'), (1002, 'efg'), (1003, 'f'),
+       (2000, 'e'), (2001, 'fff'), (2002, 'ghi'), (2003, 'h');
+update parted_trigger set a = a + 2; -- notice for odd 'a' values, long 'b' values
+NOTICE:  trigger parted_trigger on parted_trigger_1 AFTER UPDATE for ROW: (a,b)=(3,bbb)
+NOTICE:  trigger parted_trigger on parted_trigger_2 AFTER UPDATE for ROW: (a,b)=(1003,ddd)
+NOTICE:  trigger parted_trigger on parted_trigger_3_2 AFTER UPDATE for ROW: (a,b)=(2003,fff)
+drop table parted_trigger;
+-- try a constraint trigger, also
+create table parted_referenced (a int);
+create table unparted_trigger (a int, b text); -- for comparison purposes
+create table parted_trigger (a int, b text) partition by range (a);
+create table parted_trigger_1 partition of parted_trigger for values from (0) to (1000);
+create table parted_trigger_2 (drp int, a int, b text);
+alter table parted_trigger_2 drop column drp;
+alter table parted_trigger attach partition parted_trigger_2 for values from (1000) to (2000);
+create constraint trigger parted_trigger after update on parted_trigger
+  from parted_referenced
+  for each row execute procedure trigger_notice_ab();
+create constraint trigger parted_trigger after update on unparted_trigger
+  from parted_referenced
+  for each row execute procedure trigger_notice_ab();
+create table parted_trigger_3 (b text, a int) partition by range (length(b));
+create table parted_trigger_3_1 partition of parted_trigger_3 for values from (1) to (3);
+create table parted_trigger_3_2 partition of parted_trigger_3 for values from (3) to (5);
+alter table parted_trigger attach partition parted_trigger_3 for values from (2000) to (3000);
+select tgname, conname, t.tgrelid::regclass, t.tgconstrrelid::regclass,
+  c.conrelid::regclass, c.confrelid::regclass
+  from pg_trigger t join pg_constraint c on (t.tgconstraint = c.oid)
+  order by t.tgrelid::regclass::text;
+     tgname     |    conname     |      tgrelid       |   tgconstrrelid   |      conrelid      | confrelid 
+----------------+----------------+--------------------+-------------------+--------------------+-----------
+ parted_trigger | parted_trigger | parted_trigger     | parted_referenced | parted_trigger     | -
+ parted_trigger | parted_trigger | parted_trigger_1   | parted_referenced | parted_trigger_1   | -
+ parted_trigger | parted_trigger | parted_trigger_2   | parted_referenced | parted_trigger_2   | -
+ parted_trigger | parted_trigger | parted_trigger_3   | parted_referenced | parted_trigger_3   | -
+ parted_trigger | parted_trigger | parted_trigger_3_1 | parted_referenced | parted_trigger_3_1 | -
+ parted_trigger | parted_trigger | parted_trigger_3_2 | parted_referenced | parted_trigger_3_2 | -
+ parted_trigger | parted_trigger | unparted_trigger   | parted_referenced | unparted_trigger   | -
+(7 rows)
+
+drop table parted_referenced, parted_trigger, unparted_trigger;
+-- verify that the "AFTER UPDATE OF columns" event is propagated correctly
+create table parted_trigger (a int, b text) partition by range (a);
+create table parted_trigger_1 partition of parted_trigger for values from (0) to (1000);
+create table parted_trigger_2 (drp int, a int, b text);
+alter table parted_trigger_2 drop column drp;
+alter table parted_trigger attach partition parted_trigger_2 for values from (1000) to (2000);
+create trigger parted_trigger after update of b on parted_trigger
+  for each row execute procedure trigger_notice_ab();
+create table parted_trigger_3 (b text, a int) partition by range (length(b));
+create table parted_trigger_3_1 partition of parted_trigger_3 for values from (1) to (4);
+create table parted_trigger_3_2 partition of parted_trigger_3 for values from (4) to (8);
+alter table parted_trigger attach partition parted_trigger_3 for values from (2000) to (3000);
+insert into parted_trigger values (0, 'a'), (1000, 'c'), (2000, 'e'), (2001, 'eeee');
+update parted_trigger set a = a + 2;   -- no notices here
+update parted_trigger set b = b || 'b';        -- all triggers should fire
+NOTICE:  trigger parted_trigger on parted_trigger_1 AFTER UPDATE for ROW: (a,b)=(2,ab)
+NOTICE:  trigger parted_trigger on parted_trigger_2 AFTER UPDATE for ROW: (a,b)=(1002,cb)
+NOTICE:  trigger parted_trigger on parted_trigger_3_1 AFTER UPDATE for ROW: (a,b)=(2002,eb)
+NOTICE:  trigger parted_trigger on parted_trigger_3_2 AFTER UPDATE for ROW: (a,b)=(2003,eeeeb)
+drop table parted_trigger;
+drop function trigger_notice_ab();
 --
 -- Test the interaction between transition tables and both kinds of
 -- inheritance.  We'll dump the contents of the transition tables in a
index dbab8f159b8436bae7ecb56ddf299997cbfdf758..98dd4210e999d42bee9cb838273fca28918686e1 100644 (file)
@@ -394,6 +394,22 @@ SET CONSTRAINTS ALL IMMEDIATE; -- should fail
 
 COMMIT;
 
+-- test deferrable UNIQUE with a partitioned table
+CREATE TABLE parted_uniq_tbl (i int UNIQUE DEFERRABLE) partition by range (i);
+CREATE TABLE parted_uniq_tbl_1 PARTITION OF parted_uniq_tbl FOR VALUES FROM (0) TO (10);
+CREATE TABLE parted_uniq_tbl_2 PARTITION OF parted_uniq_tbl FOR VALUES FROM (20) TO (30);
+SELECT conname, conrelid::regclass FROM pg_constraint
+  WHERE conname LIKE 'parted_uniq%' ORDER BY conname;
+BEGIN;
+INSERT INTO parted_uniq_tbl VALUES (1);
+SAVEPOINT f;
+INSERT INTO parted_uniq_tbl VALUES (1);        -- unique violation
+ROLLBACK TO f;
+SET CONSTRAINTS parted_uniq_tbl_i_key DEFERRED;
+INSERT INTO parted_uniq_tbl VALUES (1);        -- OK now, fail at commit
+COMMIT;
+DROP TABLE parted_uniq_tbl;
+
 -- test a HOT update that invalidates the conflicting tuple.
 -- the trigger should still fire and catch the violation
 
index bb75165cc22d433b9ccf0cc8af88954c881e9d6a..a6a1df18e7396c1777eb1740977bb7a35afe742e 100644 (file)
@@ -547,6 +547,32 @@ SET CONSTRAINTS ALL IMMEDIATE; -- should fail
 ERROR:  duplicate key value violates unique constraint "unique_tbl_i_key"
 DETAIL:  Key (i)=(3) already exists.
 COMMIT;
+-- test deferrable UNIQUE with a partitioned table
+CREATE TABLE parted_uniq_tbl (i int UNIQUE DEFERRABLE) partition by range (i);
+CREATE TABLE parted_uniq_tbl_1 PARTITION OF parted_uniq_tbl FOR VALUES FROM (0) TO (10);
+CREATE TABLE parted_uniq_tbl_2 PARTITION OF parted_uniq_tbl FOR VALUES FROM (20) TO (30);
+SELECT conname, conrelid::regclass FROM pg_constraint
+  WHERE conname LIKE 'parted_uniq%' ORDER BY conname;
+         conname         |     conrelid      
+-------------------------+-------------------
+ parted_uniq_tbl_1_i_key | parted_uniq_tbl_1
+ parted_uniq_tbl_2_i_key | parted_uniq_tbl_2
+ parted_uniq_tbl_i_key   | parted_uniq_tbl
+(3 rows)
+
+BEGIN;
+INSERT INTO parted_uniq_tbl VALUES (1);
+SAVEPOINT f;
+INSERT INTO parted_uniq_tbl VALUES (1);        -- unique violation
+ERROR:  duplicate key value violates unique constraint "parted_uniq_tbl_1_i_key"
+DETAIL:  Key (i)=(1) already exists.
+ROLLBACK TO f;
+SET CONSTRAINTS parted_uniq_tbl_i_key DEFERRED;
+INSERT INTO parted_uniq_tbl VALUES (1);        -- OK now, fail at commit
+COMMIT;
+ERROR:  duplicate key value violates unique constraint "parted_uniq_tbl_1_i_key"
+DETAIL:  Key (i)=(1) already exists.
+DROP TABLE parted_uniq_tbl;
 -- test a HOT update that invalidates the conflicting tuple.
 -- the trigger should still fire and catch the violation
 BEGIN;
index fcf9990f6b014faf86e02bf81c0b327f3ed345d2..656cace4511eb32aa9196ca41d9714236b00921a 100644 (file)
@@ -185,6 +185,10 @@ SELECT     ctid, conindid
 FROM   pg_catalog.pg_constraint fk
 WHERE  conindid != 0 AND
        NOT EXISTS(SELECT 1 FROM pg_catalog.pg_class pk WHERE pk.oid = fk.conindid);
+SELECT ctid, conparentid
+FROM   pg_catalog.pg_constraint fk
+WHERE  conparentid != 0 AND
+       NOT EXISTS(SELECT 1 FROM pg_catalog.pg_constraint pk WHERE pk.oid = fk.conparentid);
 SELECT ctid, confrelid
 FROM   pg_catalog.pg_constraint fk
 WHERE  confrelid != 0 AND
index 3354f4899f75beef09a2e9d86a8791430f122cdf..8be893bd1e30645475e72248e18f4c97543c8a3e 100644 (file)
@@ -1286,7 +1286,46 @@ drop view my_view;
 drop table my_table;
 
 --
--- Verify that per-statement triggers are fired for partitioned tables
+-- Verify cases that are unsupported with partitioned tables
+--
+create table parted_trig (a int) partition by list (a);
+create function trigger_nothing() returns trigger
+  language plpgsql as $$ begin end; $$;
+create trigger failed before insert or update or delete on parted_trig
+  for each row execute procedure trigger_nothing();
+create trigger failed instead of update on parted_trig
+  for each row execute procedure trigger_nothing();
+create trigger failed after update on parted_trig
+  referencing old table as old_table
+  for each row execute procedure trigger_nothing();
+drop table parted_trig;
+
+--
+-- Verify trigger creation for partitioned tables, and drop behavior
+--
+create table trigpart (a int, b int) partition by range (a);
+create table trigpart1 partition of trigpart for values from (0) to (1000);
+create trigger trg1 after insert on trigpart for each row execute procedure trigger_nothing();
+create table trigpart2 partition of trigpart for values from (1000) to (2000);
+create table trigpart3 (like trigpart);
+alter table trigpart attach partition trigpart3 for values from (2000) to (3000);
+select tgrelid::regclass, tgname, tgfoid::regproc from pg_trigger
+  where tgrelid::regclass::text like 'trigpart%' order by tgrelid::regclass::text;
+drop trigger trg1 on trigpart1;        -- fail
+drop trigger trg1 on trigpart2;        -- fail
+drop trigger trg1 on trigpart3;        -- fail
+drop table trigpart2;                  -- ok, trigger should be gone in that partition
+select tgrelid::regclass, tgname, tgfoid::regproc from pg_trigger
+  where tgrelid::regclass::text like 'trigpart%' order by tgrelid::regclass::text;
+drop trigger trg1 on trigpart;         -- ok, all gone
+select tgrelid::regclass, tgname, tgfoid::regproc from pg_trigger
+  where tgrelid::regclass::text like 'trigpart%' order by tgrelid::regclass::text;
+
+drop table trigpart;
+drop function trigger_nothing();
+
+--
+-- Verify that triggers are fired for partitioned tables
 --
 create table parted_stmt_trig (a int) partition by list (a);
 create table parted_stmt_trig1 partition of parted_stmt_trig for values in (1);
@@ -1306,7 +1345,7 @@ create or replace function trigger_notice() returns trigger as $$
   end;
   $$ language plpgsql;
 
--- insert/update/delete statment-level triggers on the parent
+-- insert/update/delete statement-level triggers on the parent
 create trigger trig_ins_before before insert on parted_stmt_trig
   for each statement execute procedure trigger_notice();
 create trigger trig_ins_after after insert on parted_stmt_trig
@@ -1320,28 +1359,40 @@ create trigger trig_del_before before delete on parted_stmt_trig
 create trigger trig_del_after after delete on parted_stmt_trig
   for each statement execute procedure trigger_notice();
 
+-- insert/update/delete row-level triggers on the parent
+create trigger trig_ins_after_parent after insert on parted_stmt_trig
+  for each row execute procedure trigger_notice();
+create trigger trig_upd_after_parent after update on parted_stmt_trig
+  for each row execute procedure trigger_notice();
+create trigger trig_del_after_parent after delete on parted_stmt_trig
+  for each row execute procedure trigger_notice();
+
 -- insert/update/delete row-level triggers on the first partition
-create trigger trig_ins_before before insert on parted_stmt_trig1
+create trigger trig_ins_before_child before insert on parted_stmt_trig1
   for each row execute procedure trigger_notice();
-create trigger trig_ins_after after insert on parted_stmt_trig1
+create trigger trig_ins_after_child after insert on parted_stmt_trig1
   for each row execute procedure trigger_notice();
-create trigger trig_upd_before before update on parted_stmt_trig1
+create trigger trig_upd_before_child before update on parted_stmt_trig1
   for each row execute procedure trigger_notice();
-create trigger trig_upd_after after update on parted_stmt_trig1
+create trigger trig_upd_after_child after update on parted_stmt_trig1
+  for each row execute procedure trigger_notice();
+create trigger trig_del_before_child before delete on parted_stmt_trig1
+  for each row execute procedure trigger_notice();
+create trigger trig_del_after_child after delete on parted_stmt_trig1
   for each row execute procedure trigger_notice();
 
 -- insert/update/delete statement-level triggers on the parent
-create trigger trig_ins_before before insert on parted2_stmt_trig
+create trigger trig_ins_before_3 before insert on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_ins_after after insert on parted2_stmt_trig
+create trigger trig_ins_after_3 after insert on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_upd_before before update on parted2_stmt_trig
+create trigger trig_upd_before_3 before update on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_upd_after after update on parted2_stmt_trig
+create trigger trig_upd_after_3 after update on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_del_before before delete on parted2_stmt_trig
+create trigger trig_del_before_3 before delete on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
-create trigger trig_del_after after delete on parted2_stmt_trig
+create trigger trig_del_after_3 after delete on parted2_stmt_trig
   for each statement execute procedure trigger_notice();
 
 with ins (a) as (
@@ -1365,8 +1416,167 @@ copy parted_stmt_trig1(a) from stdin;
 1
 \.
 
+-- Disabling a trigger in the parent table should disable children triggers too
+alter table parted_stmt_trig disable trigger trig_ins_after_parent;
+insert into parted_stmt_trig values (1);
+alter table parted_stmt_trig enable trigger trig_ins_after_parent;
+insert into parted_stmt_trig values (1);
+
 drop table parted_stmt_trig, parted2_stmt_trig;
 
+-- Verify that triggers fire in alphabetical order
+create table parted_trig (a int) partition by range (a);
+create table parted_trig_1 partition of parted_trig for values from (0) to (1000)
+   partition by range (a);
+create table parted_trig_1_1 partition of parted_trig_1 for values from (0) to (100);
+create table parted_trig_2 partition of parted_trig for values from (1000) to (2000);
+create trigger zzz after insert on parted_trig for each row execute procedure trigger_notice();
+create trigger mmm after insert on parted_trig_1_1 for each row execute procedure trigger_notice();
+create trigger aaa after insert on parted_trig_1 for each row execute procedure trigger_notice();
+create trigger bbb after insert on parted_trig for each row execute procedure trigger_notice();
+create trigger qqq after insert on parted_trig_1_1 for each row execute procedure trigger_notice();
+insert into parted_trig values (50), (1500);
+drop table parted_trig;
+
+-- test irregular partitions (i.e., different column definitions),
+-- including that the WHEN clause works
+create function bark(text) returns bool language plpgsql immutable
+  as $$ begin raise notice '% <- woof!', $1; return true; end; $$;
+create or replace function trigger_notice_ab() returns trigger as $$
+  begin
+    raise notice 'trigger % on % % % for %: (a,b)=(%,%)',
+               TG_NAME, TG_TABLE_NAME, TG_WHEN, TG_OP, TG_LEVEL,
+               NEW.a, NEW.b;
+    if TG_LEVEL = 'ROW' then
+       return NEW;
+    end if;
+    return null;
+  end;
+  $$ language plpgsql;
+create table parted_irreg_ancestor (fd text, b text, fd2 int, fd3 int, a int)
+  partition by range (b);
+alter table parted_irreg_ancestor drop column fd,
+  drop column fd2, drop column fd3;
+create table parted_irreg (fd int, a int, fd2 int, b text)
+  partition by range (b);
+alter table parted_irreg drop column fd, drop column fd2;
+alter table parted_irreg_ancestor attach partition parted_irreg
+  for values from ('aaaa') to ('zzzz');
+create table parted1_irreg (b text, fd int, a int);
+alter table parted1_irreg drop column fd;
+alter table parted_irreg attach partition parted1_irreg
+  for values from ('aaaa') to ('bbbb');
+create trigger parted_trig after insert on parted_irreg
+  for each row execute procedure trigger_notice_ab();
+create trigger parted_trig_odd after insert on parted_irreg for each row
+  when (bark(new.b) AND new.a % 2 = 1) execute procedure trigger_notice_ab();
+-- we should hear barking for every insert, but parted_trig_odd only emits
+-- noise for odd values of a. parted_trig does it for all inserts.
+insert into parted_irreg values (1, 'aardvark'), (2, 'aanimals');
+insert into parted1_irreg values ('aardwolf', 2);
+insert into parted_irreg_ancestor values ('aasvogel', 3);
+drop table parted_irreg_ancestor;
+
+--
+-- Constraint triggers and partitioned tables
+create table parted_constr_ancestor (a int, b text)
+  partition by range (b);
+create table parted_constr (a int, b text)
+  partition by range (b);
+alter table parted_constr_ancestor attach partition parted_constr
+  for values from ('aaaa') to ('zzzz');
+create table parted1_constr (a int, b text);
+alter table parted_constr attach partition parted1_constr
+  for values from ('aaaa') to ('bbbb');
+create constraint trigger parted_trig after insert on parted_constr_ancestor
+  deferrable
+  for each row execute procedure trigger_notice_ab();
+create constraint trigger parted_trig_two after insert on parted_constr
+  deferrable initially deferred
+  for each row when (bark(new.b) AND new.a % 2 = 1)
+  execute procedure trigger_notice_ab();
+
+-- The immediate constraint is fired immediately; the WHEN clause of the
+-- deferred constraint is also called immediately.  The deferred constraint
+-- is fired at commit time.
+begin;
+insert into parted_constr values (1, 'aardvark');
+insert into parted1_constr values (2, 'aardwolf');
+insert into parted_constr_ancestor values (3, 'aasvogel');
+commit;
+
+-- The WHEN clause is immediate, and both constraint triggers are fired at
+-- commit time.
+begin;
+set constraints parted_trig deferred;
+insert into parted_constr values (1, 'aardvark');
+insert into parted1_constr values (2, 'aardwolf'), (3, 'aasvogel');
+commit;
+drop table parted_constr_ancestor;
+drop function bark(text);
+
+-- Test that the WHEN clause is set properly to partitions
+create table parted_trigger (a int, b text) partition by range (a);
+create table parted_trigger_1 partition of parted_trigger for values from (0) to (1000);
+create table parted_trigger_2 (drp int, a int, b text);
+alter table parted_trigger_2 drop column drp;
+alter table parted_trigger attach partition parted_trigger_2 for values from (1000) to (2000);
+create trigger parted_trigger after update on parted_trigger
+  for each row when (new.a % 2 = 1 and length(old.b) >= 2) execute procedure trigger_notice_ab();
+create table parted_trigger_3 (b text, a int) partition by range (length(b));
+create table parted_trigger_3_1 partition of parted_trigger_3 for values from (1) to (3);
+create table parted_trigger_3_2 partition of parted_trigger_3 for values from (3) to (5);
+alter table parted_trigger attach partition parted_trigger_3 for values from (2000) to (3000);
+insert into parted_trigger values
+    (0, 'a'), (1, 'bbb'), (2, 'bcd'), (3, 'c'),
+       (1000, 'c'), (1001, 'ddd'), (1002, 'efg'), (1003, 'f'),
+       (2000, 'e'), (2001, 'fff'), (2002, 'ghi'), (2003, 'h');
+update parted_trigger set a = a + 2; -- notice for odd 'a' values, long 'b' values
+drop table parted_trigger;
+
+-- try a constraint trigger, also
+create table parted_referenced (a int);
+create table unparted_trigger (a int, b text); -- for comparison purposes
+create table parted_trigger (a int, b text) partition by range (a);
+create table parted_trigger_1 partition of parted_trigger for values from (0) to (1000);
+create table parted_trigger_2 (drp int, a int, b text);
+alter table parted_trigger_2 drop column drp;
+alter table parted_trigger attach partition parted_trigger_2 for values from (1000) to (2000);
+create constraint trigger parted_trigger after update on parted_trigger
+  from parted_referenced
+  for each row execute procedure trigger_notice_ab();
+create constraint trigger parted_trigger after update on unparted_trigger
+  from parted_referenced
+  for each row execute procedure trigger_notice_ab();
+create table parted_trigger_3 (b text, a int) partition by range (length(b));
+create table parted_trigger_3_1 partition of parted_trigger_3 for values from (1) to (3);
+create table parted_trigger_3_2 partition of parted_trigger_3 for values from (3) to (5);
+alter table parted_trigger attach partition parted_trigger_3 for values from (2000) to (3000);
+select tgname, conname, t.tgrelid::regclass, t.tgconstrrelid::regclass,
+  c.conrelid::regclass, c.confrelid::regclass
+  from pg_trigger t join pg_constraint c on (t.tgconstraint = c.oid)
+  order by t.tgrelid::regclass::text;
+drop table parted_referenced, parted_trigger, unparted_trigger;
+
+-- verify that the "AFTER UPDATE OF columns" event is propagated correctly
+create table parted_trigger (a int, b text) partition by range (a);
+create table parted_trigger_1 partition of parted_trigger for values from (0) to (1000);
+create table parted_trigger_2 (drp int, a int, b text);
+alter table parted_trigger_2 drop column drp;
+alter table parted_trigger attach partition parted_trigger_2 for values from (1000) to (2000);
+create trigger parted_trigger after update of b on parted_trigger
+  for each row execute procedure trigger_notice_ab();
+create table parted_trigger_3 (b text, a int) partition by range (length(b));
+create table parted_trigger_3_1 partition of parted_trigger_3 for values from (1) to (4);
+create table parted_trigger_3_2 partition of parted_trigger_3 for values from (4) to (8);
+alter table parted_trigger attach partition parted_trigger_3 for values from (2000) to (3000);
+insert into parted_trigger values (0, 'a'), (1000, 'c'), (2000, 'e'), (2001, 'eeee');
+update parted_trigger set a = a + 2;   -- no notices here
+update parted_trigger set b = b || 'b';        -- all triggers should fire
+drop table parted_trigger;
+
+drop function trigger_notice_ab();
+
 --
 -- Test the interaction between transition tables and both kinds of
 -- inheritance.  We'll dump the contents of the transition tables in a