diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2..5a436a1 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2993,6 +2993,11 @@ VALUES ('Albany', NULL, NULL, 'NY');
     foreign table partitions.
    </para>
 
+   <para>
+    Updating the partition key of a row might cause it to be moved into a
+    different partition where this row satisfies its partition constraint.
+   </para>
+
    <sect3 id="ddl-partitioning-declarative-example">
     <title>Example</title>
 
@@ -3285,9 +3290,20 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
 
      <listitem>
       <para>
-       An <command>UPDATE</> that causes a row to move from one partition to
-       another fails, because the new value of the row fails to satisfy the
-       implicit partition constraint of the original partition.
+       When an <command>UPDATE</> causes a row to move from one partition to
+       another, there is a chance that another concurrent <command>UPDATE</> or
+       <command>DELETE</> misses this row. Suppose, during the row movement,
+       the row is still visible for the concurrent session, and it is about to
+       do an <command>UPDATE</> or <command>DELETE</> operation on the same
+       row. This DML operation can silently miss this row if the row now gets
+       deleted from the partition by the first session as part of its
+       <command>UPDATE</> row movement. In such case, the concurrent
+       <command>UPDATE</>/<command>DELETE</>, being unaware of the row
+       movement, interprets that the row has just been deleted so there is
+       nothing to be done for this row. Whereas, in the usual case where the
+       table is not partitioned, or where there is no row movement, the second
+       session would have identified the newly updated row and carried
+       <command>UPDATE</>/<command>DELETE</> on this new row version.
       </para>
      </listitem>
 
diff --git a/doc/src/sgml/ref/update.sgml b/doc/src/sgml/ref/update.sgml
index 8a1619f..28cfc1a 100644
--- a/doc/src/sgml/ref/update.sgml
+++ b/doc/src/sgml/ref/update.sgml
@@ -282,10 +282,17 @@ UPDATE <replaceable class="parameter">count</replaceable>
 
   <para>
    In the case of a partitioned table, updating a row might cause it to no
-   longer satisfy the partition constraint.  Since there is no provision to
-   move the row to the partition appropriate to the new value of its
-   partitioning key, an error will occur in this case.  This can also happen
-   when updating a partition directly.
+   longer satisfy the partition constraint of the containing partition. In that
+   case, if there is some other partition in the partition tree for which this
+   row satisfies its partition constraint, then the row is moved to that
+   partition. If there isn't such a partition, an error will occur. The error
+   will also occur when updating a partition directly. Behind the scenes, the
+   row movement is actually a <command>DELETE</> and
+   <command>INSERT</> operation. However, there is a possibility that a
+   concurrent <command>UPDATE</> or <command>DELETE</> on the same row may miss
+   this row. For details see the section
+   <xref linkend="ddl-partitioning-declarative-limitations">.
+
   </para>
  </refsect1>
 
diff --git a/doc/src/sgml/trigger.sgml b/doc/src/sgml/trigger.sgml
index 950245d..72300a0 100644
--- a/doc/src/sgml/trigger.sgml
+++ b/doc/src/sgml/trigger.sgml
@@ -160,6 +160,29 @@
    </para>
 
    <para>
+    If an <command>UPDATE</command> on a partitioned table causes a row to
+    move to another partition, it will be performed as a
+    <command>DELETE</command> from the original partition followed by
+    <command>INSERT</command> into the new partition. In this case, all
+    row-level <literal>BEFORE</> <command>UPDATE</command> triggers and all
+    row-level <literal>BEFORE</> <command>DELETE</command> triggers are fired
+    on the original partition. Then all row-level <literal>BEFORE</>
+    <command>INSERT</command> triggers are fired on the destination partition.
+    The possibility of surprising outcomes should be considered when all these
+    triggers affect the row being moved. As far as <literal>AFTER ROW</>
+    triggers are concerned, <literal>AFTER</> <command>DELETE</command> and
+    <literal>AFTER</> <command>INSERT</command> triggers are applied; but
+    <literal>AFTER</> <command>UPDATE</command> triggers are not applied
+    because the <command>UPDATE</command> has been converted to a
+    <command>DELETE</command> and <command>INSERT</command>. As far as
+    statement-level triggers are concerned, none of the
+    <command>DELETE</command> or <command>INSERT</command> triggers are fired,
+    even if row movement occurs; only the <command>UPDATE</command> triggers
+    defined on the target table used in the <command>UPDATE</command> statement
+    will be fired.
+   </para>
+
+   <para>
     Trigger functions invoked by per-statement triggers should always
     return <symbol>NULL</symbol>. Trigger functions invoked by per-row
     triggers can return a table row (a value of
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 74736e0..4bd8485 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -105,6 +105,16 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+/*
+ * List of these elements is prepared while traversing a partition tree,
+ * so as to get a consistent order of partitions.
+ */
+typedef struct ParentChild
+{
+	Oid         reloid;
+	Relation    parent;			/* Parent relation of reloid */
+} ParentChild;
+
 static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
 							   void *arg);
 static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
@@ -140,6 +150,8 @@ static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
 
+static List *append_rel_partition_oids(List *rel_list, Relation rel);
+
 /*
  * RelationBuildPartitionDesc
  *		Form rel's partition descriptor
@@ -893,7 +905,8 @@ get_qual_from_partbound(Relation rel, Relation parent,
 
 /*
  * map_partition_varattnos - maps varattno of any Vars in expr from the
- * parent attno to partition attno.
+ * attno's of 'from_rel' partition to the attno's of 'to_rel' partition.
+ * The rels can be both leaf partition or a partitioned table.
  *
  * We must allow for cases where physical attnos of a partition can be
  * different from the parent's.
@@ -903,8 +916,8 @@ get_qual_from_partbound(Relation rel, Relation parent,
  * are working on Lists, so it's less messy to do the casts internally.
  */
 List *
-map_partition_varattnos(List *expr, int target_varno,
-						Relation partrel, Relation parent)
+map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel)
 {
 	AttrNumber *part_attnos;
 	bool		found_whole_row;
@@ -912,13 +925,13 @@ map_partition_varattnos(List *expr, int target_varno,
 	if (expr == NIL)
 		return NIL;
 
-	part_attnos = convert_tuples_by_name_map(RelationGetDescr(partrel),
-											 RelationGetDescr(parent),
+	part_attnos = convert_tuples_by_name_map(RelationGetDescr(to_rel),
+											 RelationGetDescr(from_rel),
 											 gettext_noop("could not convert row type"));
 	expr = (List *) map_variable_attnos((Node *) expr,
-										target_varno, 0,
+										fromrel_varno, 0,
 										part_attnos,
-										RelationGetDescr(parent)->natts,
+										RelationGetDescr(from_rel)->natts,
 										&found_whole_row);
 	/* There can never be a whole-row reference here */
 	if (found_whole_row)
@@ -971,20 +984,9 @@ get_partition_qual_relid(Oid relid)
 	return result;
 }
 
-/*
- * Append OIDs of rel's partitions to the list 'partoids' and for each OID,
- * append pointer rel to the list 'parents'.
- */
-#define APPEND_REL_PARTITION_OIDS(rel, partoids, parents) \
-	do\
-	{\
-		int		i;\
-		for (i = 0; i < (rel)->rd_partdesc->nparts; i++)\
-		{\
-			(partoids) = lappend_oid((partoids), (rel)->rd_partdesc->oids[i]);\
-			(parents) = lappend((parents), (rel));\
-		}\
-	} while(0)
+#ifdef DEBUG_PRINT_OIDS
+static void print_oids(List *oid_list);
+#endif
 
 /*
  * RelationGetPartitionDispatchInfo
@@ -998,11 +1000,13 @@ PartitionDispatch *
 RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
 								 int *num_parted, List **leaf_part_oids)
 {
+	PartitionWalker walker;
 	PartitionDispatchData **pd;
-	List	   *all_parts = NIL,
-			   *all_parents = NIL,
-			   *parted_rels,
+	Relation	partrel;
+	Relation	parent;
+	List	   *parted_rels,
 			   *parted_rel_parents;
+	List	   *inhOIDs;
 	ListCell   *lc1,
 			   *lc2;
 	int			i,
@@ -1013,21 +1017,28 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
 	 * Lock partitions and make a list of the partitioned ones to prepare
 	 * their PartitionDispatch objects below.
 	 *
-	 * Cannot use find_all_inheritors() here, because then the order of OIDs
-	 * in parted_rels list would be unknown, which does not help, because we
-	 * assign indexes within individual PartitionDispatch in an order that is
-	 * predetermined (determined by the order of OIDs in individual partition
-	 * descriptors).
+	 * Must call find_all_inheritors() here so as to lock the partitions in a
+	 * consistent order (by oid values) to prevent deadlocks. But we assign
+	 * indexes within individual PartitionDispatch in a different order
+	 * (determined by the order of OIDs in individual partition descriptors).
+	 * So, rather than using the oids returned by find_all_inheritors(), we
+	 * generate canonically ordered oids using partition walker.
 	 */
+	inhOIDs = find_all_inheritors(RelationGetRelid(rel), lockmode, NULL);
+	list_free(inhOIDs);
+
+	partition_walker_init(&walker, rel);
+	parent = NULL;
 	*num_parted = 1;
 	parted_rels = list_make1(rel);
 	/* Root partitioned table has no parent, so NULL for parent */
 	parted_rel_parents = list_make1(NULL);
-	APPEND_REL_PARTITION_OIDS(rel, all_parts, all_parents);
-	forboth(lc1, all_parts, lc2, all_parents)
+
+	/* Go to the next partition */
+	partrel = partition_walker_next(&walker, &parent);
+
+	for (; partrel != NULL; partrel = partition_walker_next(&walker, &parent))
 	{
-		Relation	partrel = heap_open(lfirst_oid(lc1), lockmode);
-		Relation	parent = lfirst(lc2);
 		PartitionDesc partdesc = RelationGetPartitionDesc(partrel);
 
 		/*
@@ -1039,7 +1050,6 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
 			(*num_parted)++;
 			parted_rels = lappend(parted_rels, partrel);
 			parted_rel_parents = lappend(parted_rel_parents, parent);
-			APPEND_REL_PARTITION_OIDS(partrel, all_parts, all_parents);
 		}
 		else
 			heap_close(partrel, NoLock);
@@ -1143,6 +1153,10 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
 		offset += m;
 	}
 
+#ifdef DEBUG_PRINT_OIDS
+	print_oids(*leaf_part_oids);
+#endif
+
 	return pd;
 }
 
@@ -2052,6 +2066,77 @@ error_exit:
 }
 
 /*
+ * For each column of rel which is in the partition key or which appears
+ * in an expression which is in the partition key, translate the attribute
+ * number of that column according to the given parent, and add the resulting
+ * column number to the bitmapset, offset as we frequently do by
+ * FirstLowInvalidHeapAttributeNumber.
+ */
+void
+pull_child_partition_columns(Bitmapset **bitmapset,
+							 Relation rel,
+							 Relation parent)
+{
+	PartitionKey key = RelationGetPartitionKey(rel);
+	int16		partnatts = get_partition_natts(key);
+	List	   *partexprs = get_partition_exprs(key);
+	ListCell   *lc;
+	Bitmapset  *child_keycols = NULL;
+	int			i;
+	AttrNumber *map;
+	int			child_keycol = -1;
+
+	/*
+	 * First, compute the complete set of partition columns for this rel. For
+	 * compatibility with the API exposed by pull_varattnos, we offset the
+	 * column numbers by FirstLowInvalidHeapAttributeNumber.
+	 */
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+			child_keycols =
+				bms_add_member(child_keycols,
+							   partattno - FirstLowInvalidHeapAttributeNumber);
+	}
+	foreach(lc, partexprs)
+	{
+		Node	   *expr = (Node *) lfirst(lc);
+
+		pull_varattnos(expr, 1, &child_keycols);
+	}
+
+	/*
+	 * Next, work out how to convert from the attribute numbers for the child
+	 * to the attribute numbers for the parent.
+	 */
+	map =
+		convert_tuples_by_name_map(RelationGetDescr(parent),
+								   RelationGetDescr(rel),
+								   gettext_noop("could not convert row type"));
+
+	/*
+	 * For each child key column we have identified, translate to the
+	 * corresponding parent key column.  Entry 0 in the map array corresponds
+	 * to attribute number 1, which corresponds to a bitmapset entry for 1 -
+	 * FirstLowInvalidHeapAttributeNumber.
+	 */
+	while ((child_keycol = bms_next_member(child_keycols, child_keycol)) >= 0)
+	{
+		int			kc = child_keycol + FirstLowInvalidHeapAttributeNumber;
+
+		Assert(kc > 0 && kc <= RelationGetNumberOfAttributes(rel));
+		*bitmapset =
+			bms_add_member(*bitmapset,
+						   map[kc - 1] - FirstLowInvalidHeapAttributeNumber);
+	}
+
+	/* Release memory. */
+	pfree(map);
+}
+
+/*
  * qsort_partition_list_value_cmp
  *
  * Compare two list partition bound datums
@@ -2318,3 +2403,100 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+/*
+ * partition_walker_init
+ *
+ * Using the passed partitioned relation, expand it into its partitions using
+ * its partition descriptor, and make a partition rel list out of those. The
+ * rel passed in itself is not kept part of the partition list. The caller
+ * should handle the first rel separately before calling this function.
+ */
+void
+partition_walker_init(PartitionWalker *walker, Relation rel)
+{
+	memset(walker, 0, sizeof(PartitionWalker));
+
+	if (RelationGetPartitionDesc(rel))
+		walker->rels_list = append_rel_partition_oids(walker->rels_list, rel);
+
+	/* Assign the first one as the current partition cell */
+	walker->cur_cell = list_head(walker->rels_list);
+}
+
+/*
+ * partition_walker_next
+ *
+ * Get the next partition in the partition tree.
+ * At the same time, if the partition is a partitioned table, append its
+ * children at the end, so that the next time we can traverse through these.
+ */
+Relation
+partition_walker_next(PartitionWalker *walker, Relation *parent)
+{
+	ParentChild	   *pc;
+	Relation    partrel;
+
+	if (walker->cur_cell == NULL)
+		return NULL;
+
+	pc = (ParentChild *) lfirst(walker->cur_cell);
+	if (parent)
+		*parent = pc->parent;
+
+	/* Open partrel without locking; find_all_inheritors() has locked it */
+	partrel = heap_open(pc->reloid, NoLock);
+
+	/*
+	 * Append the children of partrel to the same list that we are
+	 * iterating on.
+	 */
+	if (RelationGetPartitionDesc(partrel))
+		walker->rels_list = append_rel_partition_oids(walker->rels_list,
+													  partrel);
+
+	/* Bump the cur_cell here at the end, because above, we modify the list */
+	walker->cur_cell = lnext(walker->cur_cell);
+
+	return partrel;
+}
+
+/*
+ * append_rel_partition_oids
+ *
+ * Append OIDs of rel's partitions to the list 'rel_list' and for each OID,
+ * also store parent rel.
+ */
+static
+List *append_rel_partition_oids(List *rel_list, Relation rel)
+{
+	int		i;
+	PartitionDescData *partdesc = RelationGetPartitionDesc(rel);
+
+	Assert(partdesc);
+
+	for (i = 0; i < partdesc->nparts; i++)
+	{
+		ParentChild *pc = palloc(sizeof(ParentChild));
+		pc->parent = rel;
+		pc->reloid = rel->rd_partdesc->oids[i];
+		rel_list = lappend(rel_list, pc);
+	}
+	return rel_list;
+}
+
+#ifdef DEBUG_PRINT_OIDS
+static void
+print_oids(List *oid_list)
+{
+	ListCell   *cell;
+	StringInfoData oids_str;
+
+	initStringInfo(&oids_str);
+	foreach(cell, oid_list)
+	{
+		appendStringInfo(&oids_str, "%s ", get_rel_name(lfirst_oid(cell)));
+	}
+	elog(NOTICE, "leaf oids: %s", oids_str.data);
+}
+#endif
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 53e2965..6fb3ed6 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -168,7 +168,7 @@ typedef struct CopyStateData
 	PartitionDispatch *partition_dispatch_info;
 	int			num_dispatch;	/* Number of entries in the above array */
 	int			num_partitions; /* Number of members in the following arrays */
-	ResultRelInfo *partitions;	/* Per partition result relation */
+	ResultRelInfo **partitions;	/* Per partition result relation pointers */
 	TupleConversionMap **partition_tupconv_maps;
 	TupleTableSlot *partition_tuple_slot;
 	TransitionCaptureState *transition_capture;
@@ -1426,13 +1426,15 @@ BeginCopy(ParseState *pstate,
 		if (is_from && rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 		{
 			PartitionDispatch *partition_dispatch_info;
-			ResultRelInfo *partitions;
+			ResultRelInfo **partitions;
 			TupleConversionMap **partition_tupconv_maps;
 			TupleTableSlot *partition_tuple_slot;
 			int			num_parted,
 						num_partitions;
 
 			ExecSetupPartitionTupleRouting(rel,
+										   NULL,
+										   0,
 										   1,
 										   &partition_dispatch_info,
 										   &partitions,
@@ -1462,7 +1464,7 @@ BeginCopy(ParseState *pstate,
 				for (i = 0; i < cstate->num_partitions; ++i)
 				{
 					cstate->transition_tupconv_maps[i] =
-						convert_tuples_by_name(RelationGetDescr(cstate->partitions[i].ri_RelationDesc),
+						convert_tuples_by_name(RelationGetDescr(cstate->partitions[i]->ri_RelationDesc),
 											   RelationGetDescr(rel),
 											   gettext_noop("could not convert row type"));
 				}
@@ -2609,7 +2611,7 @@ CopyFrom(CopyState cstate)
 			 * to the selected partition.
 			 */
 			saved_resultRelInfo = resultRelInfo;
-			resultRelInfo = cstate->partitions + leaf_part_index;
+			resultRelInfo = cstate->partitions[leaf_part_index];
 
 			/* We do not yet have a way to insert into a foreign partition */
 			if (resultRelInfo->ri_FdwRoutine)
@@ -2718,7 +2720,7 @@ CopyFrom(CopyState cstate)
 
 				/* Check the constraints of the tuple */
 				if (cstate->rel->rd_att->constr || check_partition_constr)
-					ExecConstraints(resultRelInfo, slot, estate);
+					ExecConstraints(resultRelInfo, slot, estate, true);
 
 				if (useHeapMultiInsert)
 				{
@@ -2838,7 +2840,7 @@ CopyFrom(CopyState cstate)
 		}
 		for (i = 0; i < cstate->num_partitions; i++)
 		{
-			ResultRelInfo *resultRelInfo = cstate->partitions + i;
+			ResultRelInfo *resultRelInfo = cstate->partitions[i];
 
 			ExecCloseIndices(resultRelInfo);
 			heap_close(resultRelInfo->ri_RelationDesc, NoLock);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index b22de78..7b22baf 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -64,6 +64,18 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+/*
+ * Entry of a temporary hash table. During UPDATE tuple routing, we want to
+ * know which of the leaf partitions are present in the UPDATE per-subplan
+ * resultRelInfo array (ModifyTableState->resultRelInfo[]). This hash table
+ * is searchable by the oids of the subplan result rels.
+ */
+typedef struct ResultRelOidsEntry
+{
+	Oid			rel_oid;
+	ResultRelInfo *resultRelInfo;
+} ResultRelOidsEntry;
+
 
 /* Hooks for plugins to get control in ExecutorStart/Run/Finish/End */
 ExecutorStart_hook_type ExecutorStart_hook = NULL;
@@ -103,8 +115,6 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 int maxfieldlen);
 static void EvalPlanQualStart(EPQState *epqstate, EState *parentestate,
 				  Plan *planTree);
-static void ExecPartitionCheck(ResultRelInfo *resultRelInfo,
-				   TupleTableSlot *slot, EState *estate);
 
 /*
  * Note that GetUpdatedColumns() also exists in commands/trigger.c.  There does
@@ -1823,15 +1833,10 @@ ExecRelCheck(ResultRelInfo *resultRelInfo,
 /*
  * ExecPartitionCheck --- check that tuple meets the partition constraint.
  */
-static void
+bool
 ExecPartitionCheck(ResultRelInfo *resultRelInfo, TupleTableSlot *slot,
 				   EState *estate)
 {
-	Relation	rel = resultRelInfo->ri_RelationDesc;
-	TupleDesc	tupdesc = RelationGetDescr(rel);
-	Bitmapset  *modifiedCols;
-	Bitmapset  *insertedCols;
-	Bitmapset  *updatedCols;
 	ExprContext *econtext;
 
 	/*
@@ -1859,51 +1864,65 @@ ExecPartitionCheck(ResultRelInfo *resultRelInfo, TupleTableSlot *slot,
 	 * As in case of the catalogued constraints, we treat a NULL result as
 	 * success here, not a failure.
 	 */
-	if (!ExecCheck(resultRelInfo->ri_PartitionCheckExpr, econtext))
-	{
-		char	   *val_desc;
-		Relation	orig_rel = rel;
+	return ExecCheck(resultRelInfo->ri_PartitionCheckExpr, econtext);
+}
 
-		/* See the comment above. */
-		if (resultRelInfo->ri_PartitionRoot)
+/*
+ * ExecPartitionCheckEmitError - Form and emit an error message after a failed
+ * partition constraint check.
+ */
+void
+ExecPartitionCheckEmitError(ResultRelInfo *resultRelInfo,
+							TupleTableSlot *slot,
+							EState *estate)
+{
+	Relation	rel = resultRelInfo->ri_RelationDesc;
+	Relation	orig_rel = rel;
+	TupleDesc	tupdesc = RelationGetDescr(rel);
+	char	   *val_desc;
+	Bitmapset  *modifiedCols;
+	Bitmapset  *insertedCols;
+	Bitmapset  *updatedCols;
+
+	/* See the comments in ExecConstraints. */
+	if (resultRelInfo->ri_PartitionRoot)
+	{
+		HeapTuple	tuple = ExecFetchSlotTuple(slot);
+		TupleDesc	old_tupdesc = RelationGetDescr(rel);
+		TupleConversionMap *map;
+
+		rel = resultRelInfo->ri_PartitionRoot;
+		tupdesc = RelationGetDescr(rel);
+		/* a reverse map */
+		map = convert_tuples_by_name(old_tupdesc, tupdesc,
+									 gettext_noop("could not convert row type"));
+		if (map != NULL)
 		{
-			HeapTuple	tuple = ExecFetchSlotTuple(slot);
-			TupleDesc	old_tupdesc = RelationGetDescr(rel);
-			TupleConversionMap *map;
-
-			rel = resultRelInfo->ri_PartitionRoot;
-			tupdesc = RelationGetDescr(rel);
-			/* a reverse map */
-			map = convert_tuples_by_name(old_tupdesc, tupdesc,
-										 gettext_noop("could not convert row type"));
-			if (map != NULL)
-			{
-				tuple = do_convert_tuple(tuple, map);
-				ExecStoreTuple(tuple, slot, InvalidBuffer, false);
-			}
+			tuple = do_convert_tuple(tuple, map);
+			ExecStoreTuple(tuple, slot, InvalidBuffer, false);
 		}
-
-		insertedCols = GetInsertedColumns(resultRelInfo, estate);
-		updatedCols = GetUpdatedColumns(resultRelInfo, estate);
-		modifiedCols = bms_union(insertedCols, updatedCols);
-		val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel),
-												 slot,
-												 tupdesc,
-												 modifiedCols,
-												 64);
-		ereport(ERROR,
-				(errcode(ERRCODE_CHECK_VIOLATION),
-				 errmsg("new row for relation \"%s\" violates partition constraint",
-						RelationGetRelationName(orig_rel)),
-				 val_desc ? errdetail("Failing row contains %s.", val_desc) : 0));
 	}
+
+	insertedCols = GetInsertedColumns(resultRelInfo, estate);
+	updatedCols = GetUpdatedColumns(resultRelInfo, estate);
+	modifiedCols = bms_union(insertedCols, updatedCols);
+	val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel),
+											 slot,
+											 tupdesc,
+											 modifiedCols,
+											 64);
+	ereport(ERROR,
+			(errcode(ERRCODE_CHECK_VIOLATION),
+			 errmsg("new row for relation \"%s\" violates partition constraint",
+					RelationGetRelationName(orig_rel)),
+			 val_desc ? errdetail("Failing row contains %s.", val_desc) : 0));
 }
 
 /*
  * ExecConstraints - check constraints of the tuple in 'slot'
  *
- * This checks the traditional NOT NULL and check constraints, as well as
- * the partition constraint, if any.
+ * This checks the traditional NOT NULL and check constraints, and if requested,
+ * checks the partition constraint.
  *
  * Note: 'slot' contains the tuple to check the constraints of, which may
  * have been converted from the original input tuple after tuple routing.
@@ -1911,7 +1930,8 @@ ExecPartitionCheck(ResultRelInfo *resultRelInfo, TupleTableSlot *slot,
  */
 void
 ExecConstraints(ResultRelInfo *resultRelInfo,
-				TupleTableSlot *slot, EState *estate)
+				TupleTableSlot *slot, EState *estate,
+				bool check_partition_constraint)
 {
 	Relation	rel = resultRelInfo->ri_RelationDesc;
 	TupleDesc	tupdesc = RelationGetDescr(rel);
@@ -2024,8 +2044,9 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 		}
 	}
 
-	if (resultRelInfo->ri_PartitionCheck)
-		ExecPartitionCheck(resultRelInfo, slot, estate);
+	if (check_partition_constraint && resultRelInfo->ri_PartitionCheck &&
+		!ExecPartitionCheck(resultRelInfo, slot, estate))
+		ExecPartitionCheckEmitError(resultRelInfo, slot, estate);
 }
 
 
@@ -2112,6 +2133,7 @@ ExecWithCheckOptions(WCOKind kind, ResultRelInfo *resultRelInfo,
 						if (map != NULL)
 						{
 							tuple = do_convert_tuple(tuple, map);
+							ExecSetSlotDescriptor(slot, tupdesc);
 							ExecStoreTuple(tuple, slot, InvalidBuffer, false);
 						}
 					}
@@ -3209,10 +3231,14 @@ EvalPlanQualEnd(EPQState *epqstate)
  * ExecSetupPartitionTupleRouting - set up information needed during
  * tuple routing for partitioned tables
  *
+ * 'update_rri' has the UPDATE per-subplan result rels.
+ * 'num_update_rri' : number of UPDATE per-subplan result rels. For INSERT,
+ *      this is 0.
+ *
  * Output arguments:
  * 'pd' receives an array of PartitionDispatch objects with one entry for
  *		every partitioned table in the partition tree
- * 'partitions' receives an array of ResultRelInfo objects with one entry for
+ * 'partitions' receives an array of ResultRelInfo* objects with one entry for
  *		every leaf partition in the partition tree
  * 'tup_conv_maps' receives an array of TupleConversionMap objects with one
  *		entry for every leaf partition (required to convert input tuple based
@@ -3232,9 +3258,11 @@ EvalPlanQualEnd(EPQState *epqstate)
  */
 void
 ExecSetupPartitionTupleRouting(Relation rel,
+							   ResultRelInfo *update_rri,
+							   int num_update_rri,
 							   Index resultRTindex,
 							   PartitionDispatch **pd,
-							   ResultRelInfo **partitions,
+							   ResultRelInfo ***partitions,
 							   TupleConversionMap ***tup_conv_maps,
 							   TupleTableSlot **partition_tuple_slot,
 							   int *num_parted, int *num_partitions)
@@ -3243,17 +3271,45 @@ ExecSetupPartitionTupleRouting(Relation rel,
 	List	   *leaf_parts;
 	ListCell   *cell;
 	int			i;
-	ResultRelInfo *leaf_part_rri;
+	ResultRelInfo *leaf_part_arr;
+	ResultRelInfo *cur_update_rri;
+	Oid			cur_reloid = InvalidOid;
 
 	/* Get the tuple-routing information and lock partitions */
 	*pd = RelationGetPartitionDispatchInfo(rel, RowExclusiveLock, num_parted,
 										   &leaf_parts);
 	*num_partitions = list_length(leaf_parts);
-	*partitions = (ResultRelInfo *) palloc(*num_partitions *
-										   sizeof(ResultRelInfo));
+	*partitions = (ResultRelInfo **) palloc(*num_partitions *
+										   sizeof(ResultRelInfo*));
 	*tup_conv_maps = (TupleConversionMap **) palloc0(*num_partitions *
 													 sizeof(TupleConversionMap *));
 
+	if (num_update_rri != 0)
+	{
+		/*
+		 * For Updates, if the leaf partition is already present in the
+		 * per-subplan result rels, we re-use that rather than initialize a new
+		 * result rel. The per-subplan resultrels and the resultrels of the
+		 * leaf partitions are both in the same canonical order. So while going
+		 * through the leaf partition oids, we need to keep track of the next
+		 * per-subplan result rel to be looked for in the leaf partition
+		 * resultrels. So, set the position of cur_update_rri to the first
+		 * per-subplan result rel, and then shift it as we find them one by one
+		 * while scanning the leaf partition oids.
+		 */
+		cur_update_rri = update_rri;
+		cur_reloid = RelationGetRelid(cur_update_rri->ri_RelationDesc);
+	}
+	else
+	{
+		/*
+		 * For inserts, we need to create all new result rels, so avoid repeated
+		 * pallocs by allocating memory for all the result rels in bulk.
+		 */
+		leaf_part_arr = (ResultRelInfo *) palloc0(*num_partitions *
+												  sizeof(ResultRelInfo));
+	}
+
 	/*
 	 * Initialize an empty slot that will be used to manipulate tuples of any
 	 * given partition's rowtype.  It is attached to the caller-specified node
@@ -3262,23 +3318,75 @@ ExecSetupPartitionTupleRouting(Relation rel,
 	 */
 	*partition_tuple_slot = MakeTupleTableSlot();
 
-	leaf_part_rri = *partitions;
 	i = 0;
 	foreach(cell, leaf_parts)
 	{
-		Relation	partrel;
+		ResultRelInfo *leaf_part_rri;
+		Relation	partrel = NULL;
 		TupleDesc	part_tupdesc;
+		Oid			leaf_oid = lfirst_oid(cell);
+
+		if (num_update_rri != 0)
+		{
+			/* Is this leaf partition present in the update resultrel ? */
+			if (cur_reloid == leaf_oid)
+			{
+				Assert(cur_update_rri <= update_rri + num_update_rri - 1);
+
+				leaf_part_rri = cur_update_rri;
+				partrel = leaf_part_rri->ri_RelationDesc;
+
+				/*
+				 * This is required when converting tuple as per root partition
+				 * tuple descriptor. When generating the update plans, this was
+				 * not set.
+				 */
+				leaf_part_rri->ri_PartitionRoot = rel;
+
+				cur_update_rri++;
+				/*
+				 * If this was the last UPDATE resultrel, indicate that by
+				 * invalidating the cur_reloid.
+				 */
+				if (cur_update_rri == update_rri + num_update_rri)
+					cur_reloid = InvalidOid;
+				else
+					cur_reloid = RelationGetRelid(cur_update_rri->ri_RelationDesc);
+			}
+			else
+				leaf_part_rri = (ResultRelInfo *) palloc0(sizeof(ResultRelInfo));
+		}
+		else
+		{
+			/* For INSERTs, we already have an array of result rels allocated */
+			leaf_part_rri = leaf_part_arr + i;
+		}
 
 		/*
-		 * We locked all the partitions above including the leaf partitions.
-		 * Note that each of the relations in *partitions are eventually
-		 * closed by the caller.
+		 * If we didn't open the partition rel, it means we haven't initialized
+		 * the result rel as well.
 		 */
-		partrel = heap_open(lfirst_oid(cell), NoLock);
+		if (!partrel)
+		{
+			/*
+			 * We locked all the partitions above including the leaf partitions.
+			 * Note that each of the newly opened relations in *partitions are
+			 * eventually closed by the caller.
+			 */
+			partrel = heap_open(leaf_oid, NoLock);
+			InitResultRelInfo(leaf_part_rri,
+							  partrel,
+							  resultRTindex,
+							  rel,
+							  0);
+		}
+
 		part_tupdesc = RelationGetDescr(partrel);
 
 		/*
-		 * Verify result relation is a valid target for the current operation.
+		 * Verify result relation is a valid target for insert operation.
+		 * Even for updates, we are doing this for tuple-routing, so again,
+		 * we need to check the validity for insert operation.
 		 */
 		CheckValidResultRel(partrel, CMD_INSERT);
 
@@ -3289,12 +3397,6 @@ ExecSetupPartitionTupleRouting(Relation rel,
 		(*tup_conv_maps)[i] = convert_tuples_by_name(tupDesc, part_tupdesc,
 													 gettext_noop("could not convert row type"));
 
-		InitResultRelInfo(leaf_part_rri,
-						  partrel,
-						  resultRTindex,
-						  rel,
-						  0);
-
 		/*
 		 * Open partition indices (remember we do not support ON CONFLICT in
 		 * case of partitioned tables, so we do not need support information
@@ -3304,9 +3406,18 @@ ExecSetupPartitionTupleRouting(Relation rel,
 			leaf_part_rri->ri_IndexRelationDescs == NULL)
 			ExecOpenIndices(leaf_part_rri, false);
 
-		leaf_part_rri++;
+		(*partitions)[i] = leaf_part_rri;
 		i++;
 	}
+
+	/*
+	 * For UPDATE, we should have found all the per-subplan resultrels in the
+	 * leaf partitions; so cur_update_rri should be positioned just next to the
+	 * last per-subplan resultrel.
+	 */
+	Assert(num_update_rri == 0 ||
+		   (cur_reloid == InvalidOid &&
+			cur_update_rri == update_rri + num_update_rri));
 }
 
 /*
@@ -3332,8 +3443,9 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
 	 * First check the root table's partition constraint, if any.  No point in
 	 * routing the tuple it if it doesn't belong in the root table itself.
 	 */
-	if (resultRelInfo->ri_PartitionCheck)
-		ExecPartitionCheck(resultRelInfo, slot, estate);
+	if (resultRelInfo->ri_PartitionCheck &&
+		!ExecPartitionCheck(resultRelInfo, slot, estate))
+		ExecPartitionCheckEmitError(resultRelInfo, slot, estate);
 
 	result = get_partition_for_tuple(pd, slot, estate,
 									 &failed_at, &failed_slot);
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index bc53d07..eca60f2 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -402,7 +402,7 @@ ExecSimpleRelationInsert(EState *estate, TupleTableSlot *slot)
 
 		/* Check the constraints of the tuple */
 		if (rel->rd_att->constr)
-			ExecConstraints(resultRelInfo, slot, estate);
+			ExecConstraints(resultRelInfo, slot, estate, true);
 
 		/* Store the slot into tuple that we can inspect. */
 		tuple = ExecMaterializeSlot(slot);
@@ -467,7 +467,7 @@ ExecSimpleRelationUpdate(EState *estate, EPQState *epqstate,
 
 		/* Check the constraints of the tuple */
 		if (rel->rd_att->constr)
-			ExecConstraints(resultRelInfo, slot, estate);
+			ExecConstraints(resultRelInfo, slot, estate, true);
 
 		/* Store the slot into tuple that we can write. */
 		tuple = ExecMaterializeSlot(slot);
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 77ba15d..9f660e7 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -45,6 +45,7 @@
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
+#include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "storage/bufmgr.h"
 #include "storage/lmgr.h"
@@ -53,6 +54,8 @@
 #include "utils/rel.h"
 #include "utils/tqual.h"
 
+#define GetUpdatedColumns(relinfo, estate) \
+	(rt_fetch((relinfo)->ri_RangeTableIndex, (estate)->es_range_table)->updatedCols)
 
 static bool ExecOnConflictUpdate(ModifyTableState *mtstate,
 					 ResultRelInfo *resultRelInfo,
@@ -239,6 +242,34 @@ ExecCheckTIDVisible(EState *estate,
 	ReleaseBuffer(buffer);
 }
 
+/*
+ * ConvertPartitionTupleSlot -- convenience function for converting tuple and
+ * storing it into a dedicated partition tuple slot. Passes the partition
+ * tuple slot back into output param p_slot. If no mapping present, keeps
+ * p_slot unchanged.
+ *
+ * Returns the converted tuple.
+ */
+static HeapTuple
+ConvertPartitionTupleSlot(ModifyTableState *mtstate, TupleConversionMap *map,
+						  HeapTuple tuple, TupleTableSlot **p_slot)
+{
+	if (!map)
+		return tuple;
+
+	tuple = do_convert_tuple(tuple, map);
+
+	/*
+	 * Change the partition tuple slot descriptor, as per converted tuple.
+	 */
+	*p_slot = mtstate->mt_partition_tuple_slot;
+	Assert(*p_slot != NULL);
+	ExecSetSlotDescriptor(*p_slot, map->outdesc);
+	ExecStoreTuple(tuple, *p_slot, InvalidBuffer, true);
+
+	return tuple;
+}
+
 /* ----------------------------------------------------------------
  *		ExecInsert
  *
@@ -280,7 +311,38 @@ ExecInsert(ModifyTableState *mtstate,
 	if (mtstate->mt_partition_dispatch_info)
 	{
 		int			leaf_part_index;
-		TupleConversionMap *map;
+		ResultRelInfo *rootResultRelInfo;
+
+		/*
+		 * If the original operation is UPDATE, the root partition rel needs to
+		 * be fetched from mtstate->rootResultRelInfo.
+		 */
+		rootResultRelInfo = (mtstate->rootResultRelInfo ?
+							 mtstate->rootResultRelInfo : resultRelInfo);
+
+		/*
+		 * If the resultRelInfo is not the root partition (which happens for
+		 * UPDATE), we should convert the tuple into root partition's tuple
+		 * descriptor, since ExecFindPartition() starts the search from root.
+		 * The tuple conversion map list is in the order of
+		 * mstate->resultRelInfo[], so to retrieve the one for this resultRel,
+		 * we need to know the position of the resultRel in
+		 * mtstate->resultRelInfo[]. Note: We assume that if the resultRelInfo
+		 * does not belong to subplans, then it already matches the root tuple
+		 * descriptor; although there is no such known scenario where this
+		 * could happen.
+		 */
+		if (rootResultRelInfo != resultRelInfo &&
+			mtstate->mt_resultrel_maps != NULL &&
+			resultRelInfo >= mtstate->resultRelInfo &&
+			resultRelInfo <= mtstate->resultRelInfo + mtstate->mt_nplans-1)
+		{
+			int		map_index = resultRelInfo - mtstate->resultRelInfo;
+
+			tuple = ConvertPartitionTupleSlot(mtstate,
+									  mtstate->mt_resultrel_maps[map_index],
+									  tuple, &slot);
+		}
 
 		/*
 		 * Away we go ... If we end up not finding a partition after all,
@@ -290,7 +352,7 @@ ExecInsert(ModifyTableState *mtstate,
 		 * the ResultRelInfo and TupleConversionMap for the partition,
 		 * respectively.
 		 */
-		leaf_part_index = ExecFindPartition(resultRelInfo,
+		leaf_part_index = ExecFindPartition(rootResultRelInfo,
 											mtstate->mt_partition_dispatch_info,
 											slot,
 											estate);
@@ -302,7 +364,7 @@ ExecInsert(ModifyTableState *mtstate,
 		 * the selected partition.
 		 */
 		saved_resultRelInfo = resultRelInfo;
-		resultRelInfo = mtstate->mt_partitions + leaf_part_index;
+		resultRelInfo = mtstate->mt_partitions[leaf_part_index];
 
 		/* We do not yet have a way to insert into a foreign partition */
 		if (resultRelInfo->ri_FdwRoutine)
@@ -347,23 +409,9 @@ ExecInsert(ModifyTableState *mtstate,
 		 * We might need to convert from the parent rowtype to the partition
 		 * rowtype.
 		 */
-		map = mtstate->mt_partition_tupconv_maps[leaf_part_index];
-		if (map)
-		{
-			Relation	partrel = resultRelInfo->ri_RelationDesc;
-
-			tuple = do_convert_tuple(tuple, map);
-
-			/*
-			 * We must use the partition's tuple descriptor from this point
-			 * on, until we're finished dealing with the partition. Use the
-			 * dedicated slot for that.
-			 */
-			slot = mtstate->mt_partition_tuple_slot;
-			Assert(slot != NULL);
-			ExecSetSlotDescriptor(slot, RelationGetDescr(partrel));
-			ExecStoreTuple(tuple, slot, InvalidBuffer, true);
-		}
+		tuple = ConvertPartitionTupleSlot(mtstate,
+						mtstate->mt_partition_tupconv_maps[leaf_part_index],
+						tuple, &slot);
 	}
 
 	resultRelationDesc = resultRelInfo->ri_RelationDesc;
@@ -481,7 +529,7 @@ ExecInsert(ModifyTableState *mtstate,
 
 		/* Check the constraints of the tuple */
 		if (resultRelationDesc->rd_att->constr || check_partition_constr)
-			ExecConstraints(resultRelInfo, slot, estate);
+			ExecConstraints(resultRelInfo, slot, estate, true);
 
 		if (onconflict != ONCONFLICT_NONE && resultRelInfo->ri_NumIndices > 0)
 		{
@@ -673,6 +721,8 @@ ExecDelete(ModifyTableState *mtstate,
 		   TupleTableSlot *planSlot,
 		   EPQState *epqstate,
 		   EState *estate,
+		   bool   *concurrently_deleted,
+		   bool process_returning,
 		   bool canSetTag)
 {
 	ResultRelInfo *resultRelInfo;
@@ -681,6 +731,9 @@ ExecDelete(ModifyTableState *mtstate,
 	HeapUpdateFailureData hufd;
 	TupleTableSlot *slot = NULL;
 
+	if (concurrently_deleted)
+		*concurrently_deleted = false;
+
 	/*
 	 * get information on the (current) result relation
 	 */
@@ -824,6 +877,8 @@ ldelete:;
 					}
 				}
 				/* tuple already deleted; nothing to do */
+				if (concurrently_deleted)
+					*concurrently_deleted = true;
 				return NULL;
 
 			default:
@@ -848,8 +903,8 @@ ldelete:;
 	ExecARDeleteTriggers(estate, resultRelInfo, tupleid, oldtuple,
 						 mtstate->mt_transition_capture);
 
-	/* Process RETURNING if present */
-	if (resultRelInfo->ri_projectReturning)
+	/* Process RETURNING if present and if requested */
+	if (process_returning && resultRelInfo->ri_projectReturning)
 	{
 		/*
 		 * We have to put the target tuple into a slot, which means first we
@@ -1038,12 +1093,51 @@ lreplace:;
 								 resultRelInfo, slot, estate);
 
 		/*
+		 * If a partition check fails, try to move the row into the right
+		 * partition.
+		 */
+		if (resultRelInfo->ri_PartitionCheck &&
+			!ExecPartitionCheck(resultRelInfo, slot, estate))
+		{
+			bool	concurrently_deleted;
+
+			/*
+			 * When an UPDATE is run with a leaf partition, we would not have
+			 * partition tuple routing setup. In that case, fail with partition
+			 * constraint violation error.
+			 */
+			if (mtstate->mt_partition_dispatch_info == NULL)
+				ExecPartitionCheckEmitError(resultRelInfo, slot, estate);
+
+			/* Do the row movement. */
+
+			/*
+			 * Skip RETURNING processing for DELETE. We want to return rows
+			 * from INSERT.
+			 */
+			ExecDelete(mtstate, tupleid, oldtuple, planSlot, epqstate, estate,
+					   &concurrently_deleted, false, false);
+
+			/*
+			 * The row was already deleted by a concurrent DELETE. So we don't
+			 * have anything to update.
+			 */
+			if (concurrently_deleted)
+				return NULL;
+
+			return ExecInsert(mtstate, slot, planSlot, NULL,
+								  ONCONFLICT_NONE, estate, canSetTag);
+		}
+
+		/*
 		 * Check the constraints of the tuple.  Note that we pass the same
 		 * slot for the orig_slot argument, because unlike ExecInsert(), no
 		 * tuple-routing is performed here, hence the slot remains unchanged.
+		 * We have already checked partition constraints above, so skip them
+		 * below.
 		 */
-		if (resultRelationDesc->rd_att->constr || resultRelInfo->ri_PartitionCheck)
-			ExecConstraints(resultRelInfo, slot, estate);
+		if (resultRelationDesc->rd_att->constr)
+			ExecConstraints(resultRelInfo, slot, estate, false);
 
 		/*
 		 * replace the heap tuple
@@ -1482,23 +1576,22 @@ ExecSetupTransitionCaptureState(ModifyTableState *mtstate, EState *estate)
 	 */
 	if (mtstate->mt_transition_capture != NULL)
 	{
-		ResultRelInfo *resultRelInfos;
+		ResultRelInfo *resultRelInfo;
 		int		numResultRelInfos;
+		bool	tuple_routing = (mtstate->mt_partition_dispatch_info != NULL);
 
 		/* Find the set of partitions so that we can find their TupleDescs. */
-		if (mtstate->mt_partition_dispatch_info != NULL)
+		if (tuple_routing)
 		{
 			/*
 			 * For INSERT via partitioned table, so we need TupleDescs based
 			 * on the partition routing table.
 			 */
-			resultRelInfos = mtstate->mt_partitions;
 			numResultRelInfos = mtstate->mt_num_partitions;
 		}
 		else
 		{
 			/* Otherwise we need the ResultRelInfo for each subplan. */
-			resultRelInfos = mtstate->resultRelInfo;
 			numResultRelInfos = mtstate->mt_nplans;
 		}
 
@@ -1512,8 +1605,15 @@ ExecSetupTransitionCaptureState(ModifyTableState *mtstate, EState *estate)
 			palloc0(sizeof(TupleConversionMap *) * numResultRelInfos);
 		for (i = 0; i < numResultRelInfos; ++i)
 		{
+			/*
+			 * As stated above, mapping source is different for INSERT or
+			 * otherwise.
+			 */
+			resultRelInfo = (tuple_routing ?
+					mtstate->mt_partitions[i] : &mtstate->resultRelInfo[i]);
+
 			mtstate->mt_transition_tupconv_maps[i] =
-				convert_tuples_by_name(RelationGetDescr(resultRelInfos[i].ri_RelationDesc),
+				convert_tuples_by_name(RelationGetDescr(resultRelInfo->ri_RelationDesc),
 									   RelationGetDescr(targetRelInfo->ri_RelationDesc),
 									   gettext_noop("could not convert row type"));
 		}
@@ -1746,7 +1846,8 @@ ExecModifyTable(ModifyTableState *node)
 				break;
 			case CMD_DELETE:
 				slot = ExecDelete(node, tupleid, oldtuple, planSlot,
-								  &node->mt_epqstate, estate, node->canSetTag);
+								  &node->mt_epqstate, estate,
+								  NULL, true, node->canSetTag);
 				break;
 			default:
 				elog(ERROR, "unknown operation");
@@ -1791,9 +1892,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	ResultRelInfo *resultRelInfo;
 	TupleDesc	tupDesc;
 	Plan	   *subplan;
+	int			firstVarno = 0;
+	Relation	firstResultRel = NULL;
 	ListCell   *l;
 	int			i;
 	Relation	rel;
+	bool		update_tuple_routing_needed = node->part_cols_updated;
 
 	/* check for unsupported flags */
 	Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
@@ -1865,6 +1969,15 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 			resultRelInfo->ri_IndexRelationDescs == NULL)
 			ExecOpenIndices(resultRelInfo, mtstate->mt_onconflict != ONCONFLICT_NONE);
 
+		/*
+		 * If this is an UPDATE and a BEFORE UPDATE trigger is present, we may
+		 * need to do update tuple routing.
+		 */
+		if (resultRelInfo->ri_TrigDesc &&
+			resultRelInfo->ri_TrigDesc->trig_update_before_row &&
+			operation == CMD_UPDATE)
+			update_tuple_routing_needed = true;
+
 		/* Now init the plan for this result rel */
 		estate->es_result_relation_info = resultRelInfo;
 		mtstate->mt_plans[i] = ExecInitNode(subplan, estate, eflags);
@@ -1902,18 +2015,28 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	else
 		rel = mtstate->resultRelInfo->ri_RelationDesc;
 
-	/* Build state for INSERT tuple routing */
-	if (operation == CMD_INSERT &&
-		rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+	/* Decide whether we need to perform update tuple routing. */
+	if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		update_tuple_routing_needed = false;
+
+	/*
+	 * Build state for tuple routing if it's an INSERT or if it's an UPDATE of
+	 * partition key.
+	 */
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
+		(operation == CMD_INSERT || update_tuple_routing_needed))
 	{
 		PartitionDispatch *partition_dispatch_info;
-		ResultRelInfo *partitions;
+		ResultRelInfo **partitions;
 		TupleConversionMap **partition_tupconv_maps;
 		TupleTableSlot *partition_tuple_slot;
 		int			num_parted,
 					num_partitions;
 
 		ExecSetupPartitionTupleRouting(rel,
+									   (operation == CMD_UPDATE ?
+											mtstate->resultRelInfo : NULL),
+									   (operation == CMD_UPDATE ? nplans : 0),
 									   node->nominalRelation,
 									   &partition_dispatch_info,
 									   &partitions,
@@ -1926,6 +2049,43 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		mtstate->mt_num_partitions = num_partitions;
 		mtstate->mt_partition_tupconv_maps = partition_tupconv_maps;
 		mtstate->mt_partition_tuple_slot = partition_tuple_slot;
+
+		/*
+		 * Below are required as reference objects for mapping partition
+		 * attno's in expressions such as WCO and RETURNING.
+		 */
+		firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
+		firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+	}
+
+	/*
+	 * Construct mapping from each of the resultRelInfo attnos to the root
+	 * attno. This is required when during update row movement the tuple
+	 * descriptor of a source partition does not match the root partition
+	 * descriptor. In such case we need to convert tuples to the root partition
+	 * tuple descriptor, because the search for destination partition starts
+	 * from the root. Skip this setup if it's not a partition key update or if
+	 * there are no partitions below this partitioned table.
+	 */
+	if (update_tuple_routing_needed && mtstate->mt_num_partitions > 0)
+	{
+		TupleConversionMap **tup_conv_maps;
+		TupleDesc		outdesc;
+
+		mtstate->mt_resultrel_maps =
+		(TupleConversionMap **) palloc0(sizeof(TupleConversionMap*) * nplans);
+
+		/* Get tuple descriptor of the root partition. */
+		outdesc = RelationGetDescr(mtstate->mt_partition_dispatch_info[0]->reldesc);
+
+		resultRelInfo = mtstate->resultRelInfo;
+		tup_conv_maps = mtstate->mt_resultrel_maps;
+		for (i = 0; i < nplans; i++)
+		{
+			TupleDesc indesc = RelationGetDescr(resultRelInfo[i].ri_RelationDesc);
+			tup_conv_maps[i] = convert_tuples_by_name(indesc, outdesc,
+								 gettext_noop("could not convert row type"));
+		}
 	}
 
 	/* Build state for collecting transition tuples */
@@ -1961,50 +2121,52 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	 * Build WITH CHECK OPTION constraints for each leaf partition rel. Note
 	 * that we didn't build the withCheckOptionList for each partition within
 	 * the planner, but simple translation of the varattnos for each partition
-	 * will suffice.  This only occurs for the INSERT case; UPDATE/DELETE
-	 * cases are handled above.
+	 * will suffice.  This only occurs for the INSERT case or for UPDATE
+	 * row movement. DELETEs and local UPDATEs are handled above.
 	 */
 	if (node->withCheckOptionLists != NIL && mtstate->mt_num_partitions > 0)
 	{
-		List	   *wcoList;
-		PlanState  *plan;
+		List	   *firstWco;
 
 		/*
 		 * In case of INSERT on partitioned tables, there is only one plan.
 		 * Likewise, there is only one WITH CHECK OPTIONS list, not one per
-		 * partition.  We make a copy of the WCO qual for each partition; note
-		 * that, if there are SubPlans in there, they all end up attached to
-		 * the one parent Plan node.
+		 * partition. Whereas for UPDATE, there are as many WCOs as there are
+		 * plans. So in either case, use the WCO expression of the first
+		 * resultRelInfo as a reference to calculate attno's for the WCO
+		 * expression of each of the partitions. We make a copy of the WCO qual
+		 * for each partition. Note that, if there are SubPlans in there, they
+		 * all end up attached to the one parent Plan node.
 		 */
-		Assert(operation == CMD_INSERT &&
+		Assert(update_tuple_routing_needed ||
+			   (operation == CMD_INSERT &&
 			   list_length(node->withCheckOptionLists) == 1 &&
-			   mtstate->mt_nplans == 1);
-		wcoList = linitial(node->withCheckOptionLists);
-		plan = mtstate->mt_plans[0];
-		resultRelInfo = mtstate->mt_partitions;
+			   mtstate->mt_nplans == 1));
+
+		firstWco = linitial(node->withCheckOptionLists);
 		for (i = 0; i < mtstate->mt_num_partitions; i++)
 		{
-			Relation	partrel = resultRelInfo->ri_RelationDesc;
-			List	   *mapped_wcoList;
+			Relation	partrel;
+			List	   *mappedWco;
 			List	   *wcoExprs = NIL;
 			ListCell   *ll;
 
-			/* varno = node->nominalRelation */
-			mapped_wcoList = map_partition_varattnos(wcoList,
-													 node->nominalRelation,
-													 partrel, rel);
-			foreach(ll, mapped_wcoList)
+			resultRelInfo = mtstate->mt_partitions[i];
+
+			partrel = resultRelInfo->ri_RelationDesc;
+			mappedWco = map_partition_varattnos(firstWco, firstVarno,
+												partrel, firstResultRel);
+			foreach(ll, mappedWco)
 			{
 				WithCheckOption *wco = castNode(WithCheckOption, lfirst(ll));
 				ExprState  *wcoExpr = ExecInitQual(castNode(List, wco->qual),
-												   plan);
+												   &mtstate->ps);
 
 				wcoExprs = lappend(wcoExprs, wcoExpr);
 			}
 
-			resultRelInfo->ri_WithCheckOptions = mapped_wcoList;
+			resultRelInfo->ri_WithCheckOptions = mappedWco;
 			resultRelInfo->ri_WithCheckOptionExprs = wcoExprs;
-			resultRelInfo++;
 		}
 	}
 
@@ -2015,7 +2177,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	{
 		TupleTableSlot *slot;
 		ExprContext *econtext;
-		List	   *returningList;
+		List	   *firstReturningList;
 
 		/*
 		 * Initialize result tuple slot and assign its rowtype using the first
@@ -2052,20 +2214,25 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		 * Build a projection for each leaf partition rel.  Note that we
 		 * didn't build the returningList for each partition within the
 		 * planner, but simple translation of the varattnos for each partition
-		 * will suffice.  This only occurs for the INSERT case; UPDATE/DELETE
-		 * are handled above.
+		 * will suffice.  This only occurs for the INSERT case or for UPDATE
+		 * row movement. DELETEs and local UPDATEs are handled above.
 		 */
-		resultRelInfo = mtstate->mt_partitions;
-		returningList = linitial(node->returningLists);
+		firstReturningList = linitial(node->returningLists);
 		for (i = 0; i < mtstate->mt_num_partitions; i++)
 		{
-			Relation	partrel = resultRelInfo->ri_RelationDesc;
+			Relation	partrel;
 			List	   *rlist;
 
-			/* varno = node->nominalRelation */
-			rlist = map_partition_varattnos(returningList,
-											node->nominalRelation,
-											partrel, rel);
+			resultRelInfo = mtstate->mt_partitions[i];
+			partrel = resultRelInfo->ri_RelationDesc;
+
+			/*
+			 * Use the returning expression of the first resultRelInfo as a
+			 * reference to calculate attno's for the returning expression of
+			 * each of the partitions.
+			 */
+			rlist = map_partition_varattnos(firstReturningList, firstVarno,
+											partrel, firstResultRel);
 			resultRelInfo->ri_projectReturning =
 				ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps,
 										resultRelInfo->ri_RelationDesc->rd_att);
@@ -2308,6 +2475,7 @@ void
 ExecEndModifyTable(ModifyTableState *node)
 {
 	int			i;
+	CmdType		operation = node->operation;
 
 	/* Free transition tables */
 	if (node->mt_transition_capture != NULL)
@@ -2344,7 +2512,17 @@ ExecEndModifyTable(ModifyTableState *node)
 	}
 	for (i = 0; i < node->mt_num_partitions; i++)
 	{
-		ResultRelInfo *resultRelInfo = node->mt_partitions + i;
+		ResultRelInfo *resultRelInfo = node->mt_partitions[i];
+
+		/*
+		 * If this result rel is one of the subplan result rels, let
+		 * ExecEndPlan() close it. For INSERTs, this does not apply because
+		 * all leaf partition result rels are anyway newly allocated.
+		 */
+		if (operation == CMD_UPDATE &&
+			resultRelInfo >= node->resultRelInfo &&
+			resultRelInfo < node->resultRelInfo + node->mt_nplans)
+			continue;
 
 		ExecCloseIndices(resultRelInfo);
 		heap_close(resultRelInfo->ri_RelationDesc, NoLock);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 45a04b0..4156e02 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -204,6 +204,7 @@ _copyModifyTable(const ModifyTable *from)
 	COPY_SCALAR_FIELD(canSetTag);
 	COPY_SCALAR_FIELD(nominalRelation);
 	COPY_NODE_FIELD(partitioned_rels);
+	COPY_SCALAR_FIELD(part_cols_updated);
 	COPY_NODE_FIELD(resultRelations);
 	COPY_SCALAR_FIELD(resultRelIndex);
 	COPY_SCALAR_FIELD(rootResultRelIndex);
@@ -2256,6 +2257,7 @@ _copyPartitionedChildRelInfo(const PartitionedChildRelInfo *from)
 
 	COPY_SCALAR_FIELD(parent_relid);
 	COPY_NODE_FIELD(child_rels);
+	COPY_BITMAPSET_FIELD(all_part_cols);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8d92c03..f2df72b 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -909,6 +909,7 @@ _equalPartitionedChildRelInfo(const PartitionedChildRelInfo *a, const Partitione
 {
 	COMPARE_SCALAR_FIELD(parent_relid);
 	COMPARE_NODE_FIELD(child_rels);
+	COMPARE_BITMAPSET_FIELD(all_part_cols);
 
 	return true;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 379d92a..2ca8a71 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -367,6 +367,7 @@ _outModifyTable(StringInfo str, const ModifyTable *node)
 	WRITE_BOOL_FIELD(canSetTag);
 	WRITE_UINT_FIELD(nominalRelation);
 	WRITE_NODE_FIELD(partitioned_rels);
+	WRITE_BOOL_FIELD(part_cols_updated);
 	WRITE_NODE_FIELD(resultRelations);
 	WRITE_INT_FIELD(resultRelIndex);
 	WRITE_INT_FIELD(rootResultRelIndex);
@@ -2094,6 +2095,7 @@ _outModifyTablePath(StringInfo str, const ModifyTablePath *node)
 	WRITE_BOOL_FIELD(canSetTag);
 	WRITE_UINT_FIELD(nominalRelation);
 	WRITE_NODE_FIELD(partitioned_rels);
+	WRITE_BOOL_FIELD(part_cols_updated);
 	WRITE_NODE_FIELD(resultRelations);
 	WRITE_NODE_FIELD(subpaths);
 	WRITE_NODE_FIELD(subroots);
@@ -2516,6 +2518,7 @@ _outPartitionedChildRelInfo(StringInfo str, const PartitionedChildRelInfo *node)
 
 	WRITE_UINT_FIELD(parent_relid);
 	WRITE_NODE_FIELD(child_rels);
+	WRITE_BITMAPSET_FIELD(all_part_cols);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 86c811d..949053c 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1562,6 +1562,7 @@ _readModifyTable(void)
 	READ_BOOL_FIELD(canSetTag);
 	READ_UINT_FIELD(nominalRelation);
 	READ_NODE_FIELD(partitioned_rels);
+	READ_BOOL_FIELD(part_cols_updated);
 	READ_NODE_FIELD(resultRelations);
 	READ_INT_FIELD(resultRelIndex);
 	READ_INT_FIELD(rootResultRelIndex);
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index f087ddb..064af0f 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1291,7 +1291,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	rte = planner_rt_fetch(rel->relid, root);
 	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, rel->relid);
+		partitioned_rels = get_partitioned_child_rels(root, rel->relid, NULL);
 		/* The root partitioned table is included as a child rel */
 		Assert(list_length(partitioned_rels) >= 1);
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index e589d92..7e4f058 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -277,6 +277,7 @@ static ProjectSet *make_project_set(List *tlist, Plan *subplan);
 static ModifyTable *make_modifytable(PlannerInfo *root,
 				 CmdType operation, bool canSetTag,
 				 Index nominalRelation, List *partitioned_rels,
+				 bool part_cols_updated,
 				 List *resultRelations, List *subplans,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict, int epqParam);
@@ -2357,6 +2358,7 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 							best_path->canSetTag,
 							best_path->nominalRelation,
 							best_path->partitioned_rels,
+							best_path->part_cols_updated,
 							best_path->resultRelations,
 							subplans,
 							best_path->withCheckOptionLists,
@@ -6398,6 +6400,7 @@ static ModifyTable *
 make_modifytable(PlannerInfo *root,
 				 CmdType operation, bool canSetTag,
 				 Index nominalRelation, List *partitioned_rels,
+				 bool part_cols_updated,
 				 List *resultRelations, List *subplans,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict, int epqParam)
@@ -6424,6 +6427,7 @@ make_modifytable(PlannerInfo *root,
 	node->canSetTag = canSetTag;
 	node->nominalRelation = nominalRelation;
 	node->partitioned_rels = partitioned_rels;
+	node->part_cols_updated = part_cols_updated;
 	node->resultRelations = resultRelations;
 	node->resultRelIndex = -1;	/* will be set correctly in setrefs.c */
 	node->rootResultRelIndex = -1;	/* will be set correctly in setrefs.c */
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 2988c11..cf91907 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1042,6 +1042,7 @@ inheritance_planner(PlannerInfo *root)
 	Index		rti;
 	RangeTblEntry *parent_rte;
 	List	   *partitioned_rels = NIL;
+	bool		part_cols_updated = false;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1356,9 +1357,15 @@ inheritance_planner(PlannerInfo *root)
 
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, parentRTindex);
+		Bitmapset  *all_part_cols = NULL;
+
+		partitioned_rels = get_partitioned_child_rels(root, parentRTindex,
+													  &all_part_cols);
 		/* The root partitioned table is included as a child rel */
 		Assert(list_length(partitioned_rels) >= 1);
+
+		if (bms_overlap(all_part_cols, parent_rte->updatedCols))
+			part_cols_updated = true;
 	}
 
 	/* Result path must go into outer query's FINAL upperrel */
@@ -1415,6 +1422,7 @@ inheritance_planner(PlannerInfo *root)
 									 parse->canSetTag,
 									 nominalRelation,
 									 partitioned_rels,
+									 part_cols_updated,
 									 resultRelations,
 									 subpaths,
 									 subroots,
@@ -2032,6 +2040,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
 										parse->canSetTag,
 										parse->resultRelation,
 										NIL,
+										false,
 										list_make1_int(parse->resultRelation),
 										list_make1(path),
 										list_make1(root),
@@ -6062,10 +6071,15 @@ plan_cluster_use_sort(Oid tableOid, Oid indexOid)
  *		Returns a list of the RT indexes of the partitioned child relations
  *		with rti as the root parent RT index.
  *
+ * If all_part_cols_p is non-NULL, *all_part_cols_p is set to a bitmapset
+ * of all partitioning columns used by the partitioned table or any
+ * descendent.
+ *
  * Note: Only call this function on RTEs known to be partitioned tables.
  */
 List *
-get_partitioned_child_rels(PlannerInfo *root, Index rti)
+get_partitioned_child_rels(PlannerInfo *root, Index rti,
+						   Bitmapset **all_part_cols_p)
 {
 	List	   *result = NIL;
 	ListCell   *l;
@@ -6077,6 +6091,8 @@ get_partitioned_child_rels(PlannerInfo *root, Index rti)
 		if (pc->parent_relid == rti)
 		{
 			result = pc->child_rels;
+			if (all_part_cols_p)
+				*all_part_cols_p = pc->all_part_cols;
 			break;
 		}
 	}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index cf46b74..b854d6c 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,6 +33,7 @@
 #include "access/heapam.h"
 #include "access/htup_details.h"
 #include "access/sysattr.h"
+#include "catalog/partition.h"
 #include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_type.h"
 #include "miscadmin.h"
@@ -99,6 +100,8 @@ static List *generate_append_tlist(List *colTypes, List *colCollations,
 static List *generate_setop_grouplist(SetOperationStmt *op, List *targetlist);
 static void expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte,
 						 Index rti);
+static Relation get_next_child(Relation oldrelation, ListCell **cell,
+						PartitionWalker *walker, bool is_partitioned);
 static void make_inh_translation_list(Relation oldrelation,
 						  Relation newrelation,
 						  Index newvarno,
@@ -1370,13 +1373,17 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	Oid			parentOID;
 	PlanRowMark *oldrc;
 	Relation	oldrelation;
+	Relation	newrelation;
 	LOCKMODE	lockmode;
 	List	   *inhOIDs;
 	List	   *appinfos;
-	ListCell   *l;
+	ListCell   *oids_cell;
 	bool		need_append;
+	bool		is_partitioned_resultrel;
 	PartitionedChildRelInfo *pcinfo;
+	PartitionWalker walker;
 	List	   *partitioned_child_rels = NIL;
+	Bitmapset  *all_part_cols = NULL;
 
 	/* Does RT entry allow inheritance? */
 	if (!rte->inh)
@@ -1446,23 +1453,54 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	 */
 	oldrelation = heap_open(parentOID, NoLock);
 
+	/*
+	 * Remember whether it is a result relation and it is partitioned. We need
+	 * to decide the ordering of result rels based on this.
+	 */
+	is_partitioned_resultrel =
+		(oldrelation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE
+		 && rti == parse->resultRelation);
+
 	/* Scan the inheritance set and expand it */
 	appinfos = NIL;
 	need_append = false;
-	foreach(l, inhOIDs)
+	newrelation = oldrelation;
+
+	/* For non-partitioned result-rels, open the first child from inhOIDs */
+	if (!is_partitioned_resultrel)
+	{
+		oids_cell = list_head(inhOIDs);
+		newrelation = get_next_child(oldrelation, &oids_cell, &walker,
+									 is_partitioned_resultrel);
+	}
+	else
+	{
+		/*
+		 * For partitioned resultrels, we don't need the inhOIDs list itself,
+		 * because we anyways traverse the tree in canonical order; but we do
+		 * want to lock all the children in a consistent order (see
+		 * find_inheritance_children), so as to avoid unnecessary deadlocks.
+		 * Hence, the call to find_all_inheritors() above. The aim is to
+		 * generate the appinfos in canonical order so that the result rels,
+		 * if generated later, are in the same order as those of the leaf
+		 * partitions that are maintained during insert/update tuple routing.
+		 * Maintaining same order would speed up searching for a given leaf
+		 * partition in these result rels.
+		 */
+		list_free(inhOIDs);
+		inhOIDs = NIL;
+		partition_walker_init(&walker, oldrelation);
+	}
+
+	for (; newrelation != NULL;
+		 newrelation = get_next_child(oldrelation, &oids_cell, &walker,
+									  is_partitioned_resultrel))
 	{
-		Oid			childOID = lfirst_oid(l);
-		Relation	newrelation;
+		Oid			childOID = RelationGetRelid(newrelation);
 		RangeTblEntry *childrte;
 		Index		childRTindex;
 		AppendRelInfo *appinfo;
 
-		/* Open rel if needed; we already have required locks */
-		if (childOID != parentOID)
-			newrelation = heap_open(childOID, NoLock);
-		else
-			newrelation = oldrelation;
-
 		/*
 		 * It is possible that the parent table has children that are temp
 		 * tables of other backends.  We cannot safely access such tables
@@ -1535,8 +1573,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 			}
 		}
 		else
+		{
 			partitioned_child_rels = lappend_int(partitioned_child_rels,
 												 childRTindex);
+			pull_child_partition_columns(&all_part_cols, newrelation,
+										 oldrelation);
+		}
 
 		/*
 		 * Build a PlanRowMark if parent is marked FOR UPDATE/SHARE.
@@ -1575,6 +1617,10 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 
 	heap_close(oldrelation, NoLock);
 
+#ifdef DEBUG_PRINT_OIDS
+	print_oids(appinfos, parse->rtable);
+#endif
+
 	/*
 	 * If all the children were temp tables or a partitioned parent did not
 	 * have any leaf partitions, pretend it's a non-inheritance situation; we
@@ -1604,6 +1650,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		Assert(rte->relkind == RELKIND_PARTITIONED_TABLE);
 		pcinfo->parent_relid = rti;
 		pcinfo->child_rels = partitioned_child_rels;
+		pcinfo->all_part_cols = all_part_cols;
 		root->pcinfo_list = lappend(root->pcinfo_list, pcinfo);
 	}
 
@@ -1612,6 +1659,45 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 }
 
 /*
+ * Get the next child in an inheritance tree.
+ *
+ * This function is called to traverse two different types of lists. If it's a
+ * list containing partitions, is_partitioned is true, and 'walker' is valid.
+ * Otherwise, 'cell' points to a position in the list of inheritance children.
+ * For partitions walker, the partition traversal is done using canonical
+ * ordering. Whereas, for inheritence children, list is already prepared, and
+ * is ordered depending upon the pg_inherit scan.
+ *
+ * oldrelation is the root relation in the inheritence tree. This is unused in
+ * case of is_partitioned=true.
+ */
+static Relation
+get_next_child(Relation oldrelation, ListCell **cell, PartitionWalker *walker,
+			   bool is_partitioned)
+{
+	if (is_partitioned)
+		return partition_walker_next(walker, NULL);
+	else
+	{
+		Oid		childOID;
+
+		if (!*cell)
+			return NULL; /* We are done with the list */
+
+		childOID = lfirst_oid(*cell);
+
+		/* Prepare to get the next child. */
+		*cell = lnext(*cell);
+
+		/* If it's the root relation, it is already open */
+		if (childOID != RelationGetRelid(oldrelation))
+			return heap_open(childOID, NoLock);
+		else
+			return oldrelation;
+	}
+}
+
+/*
  * make_inh_translation_list
  *	  Build the list of translations from parent Vars to child Vars for
  *	  an inheritance child.
@@ -2161,3 +2247,21 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	/* Now translate for this child */
 	return adjust_appendrel_attrs(root, node, appinfo);
 }
+
+#ifdef DEBUG_PRINT_OIDS
+static void
+print_oids(List *oid_list, List *rtable)
+{
+	ListCell   *cell;
+	StringInfoData oids_str;
+
+	initStringInfo(&oids_str);
+	foreach(cell, oid_list)
+	{
+		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(cell);
+		RangeTblEntry *childrte = (RangeTblEntry *) list_nth(rtable, appinfo->child_relid-1);
+		appendStringInfo(&oids_str, "%s ", get_rel_name(childrte->relid));
+	}
+	elog(NOTICE, "expanded oids: %s", oids_str.data);
+}
+#endif
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f2d6385..f63edf4 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3161,6 +3161,8 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
  * 'partitioned_rels' is an integer list of RT indexes of non-leaf tables in
  *		the partition tree, if this is an UPDATE/DELETE to a partitioned table.
  *		Otherwise NIL.
+ * 'part_cols_updated' if any partitioning columns are being updated, either
+ *		from the named relation or a descendent partitione table.
  * 'resultRelations' is an integer list of actual RT indexes of target rel(s)
  * 'subpaths' is a list of Path(s) producing source data (one per rel)
  * 'subroots' is a list of PlannerInfo structs (one per rel)
@@ -3174,6 +3176,7 @@ ModifyTablePath *
 create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
 						CmdType operation, bool canSetTag,
 						Index nominalRelation, List *partitioned_rels,
+						bool part_cols_updated,
 						List *resultRelations, List *subpaths,
 						List *subroots,
 						List *withCheckOptionLists, List *returningLists,
@@ -3241,6 +3244,7 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
 	pathnode->canSetTag = canSetTag;
 	pathnode->nominalRelation = nominalRelation;
 	pathnode->partitioned_rels = list_copy(partitioned_rels);
+	pathnode->part_cols_updated = part_cols_updated;
 	pathnode->resultRelations = resultRelations;
 	pathnode->subpaths = subpaths;
 	pathnode->subroots = subroots;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index f10879a..e6af17d 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -68,6 +68,12 @@ typedef struct PartitionDispatchData
 	int		   *indexes;
 } PartitionDispatchData;
 
+typedef struct PartitionWalker
+{
+	List	   *rels_list;
+	ListCell   *cur_cell;
+} PartitionWalker;
+
 typedef struct PartitionDispatchData *PartitionDispatch;
 
 extern void RelationBuildPartitionDesc(Relation relation);
@@ -79,11 +85,15 @@ extern void check_new_partition_bound(char *relname, Relation parent,
 extern Oid	get_partition_parent(Oid relid);
 extern List *get_qual_from_partbound(Relation rel, Relation parent,
 						PartitionBoundSpec *spec);
-extern List *map_partition_varattnos(List *expr, int target_varno,
-						Relation partrel, Relation parent);
+extern List *map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel);
 extern List *RelationGetPartitionQual(Relation rel);
 extern Expr *get_partition_qual_relid(Oid relid);
 
+extern void partition_walker_init(PartitionWalker *walker, Relation rel);
+extern Relation partition_walker_next(PartitionWalker *walker,
+									  Relation *parent);
+
 /* For tuple routing */
 extern PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
 								 int lockmode, int *num_parted,
@@ -98,4 +108,8 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 						EState *estate,
 						PartitionDispatchData **failed_at,
 						TupleTableSlot **failed_slot);
+extern void pull_child_partition_columns(Bitmapset **bitmapset,
+							 Relation rel,
+							 Relation parent);
+
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 59c28b7..94f8acf 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -187,7 +187,10 @@ extern ResultRelInfo *ExecGetTriggerResultRel(EState *estate, Oid relid);
 extern void ExecCleanUpTriggerState(EState *estate);
 extern bool ExecContextForcesOids(PlanState *planstate, bool *hasoids);
 extern void ExecConstraints(ResultRelInfo *resultRelInfo,
-				TupleTableSlot *slot, EState *estate);
+				TupleTableSlot *slot, EState *estate,
+				bool check_partition_constraint);
+extern void ExecPartitionCheckEmitError(ResultRelInfo *resultRelInfo,
+									TupleTableSlot *slot, EState *estate);
 extern void ExecWithCheckOptions(WCOKind kind, ResultRelInfo *resultRelInfo,
 					 TupleTableSlot *slot, EState *estate);
 extern LockTupleMode ExecUpdateLockMode(EState *estate, ResultRelInfo *relinfo);
@@ -207,9 +210,11 @@ extern void EvalPlanQualSetTuple(EPQState *epqstate, Index rti,
 					 HeapTuple tuple);
 extern HeapTuple EvalPlanQualGetTuple(EPQState *epqstate, Index rti);
 extern void ExecSetupPartitionTupleRouting(Relation rel,
+							   ResultRelInfo *update_rri,
+							   int num_update_rri,
 							   Index resultRTindex,
 							   PartitionDispatch **pd,
-							   ResultRelInfo **partitions,
+							   ResultRelInfo ***partitions,
 							   TupleConversionMap ***tup_conv_maps,
 							   TupleTableSlot **partition_tuple_slot,
 							   int *num_parted, int *num_partitions);
@@ -217,6 +222,8 @@ extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
 				  PartitionDispatch *pd,
 				  TupleTableSlot *slot,
 				  EState *estate);
+extern bool ExecPartitionCheck(ResultRelInfo *resultRelInfo,
+							TupleTableSlot *slot, EState *estate);
 
 #define EvalPlanQualSetSlot(epqstate, slot)  ((epqstate)->origslot = (slot))
 extern void EvalPlanQualFetchRowMarks(EPQState *epqstate);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 85fac8a..276b65b 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -959,9 +959,13 @@ typedef struct ModifyTableState
 	int			mt_num_dispatch;	/* Number of entries in the above array */
 	int			mt_num_partitions;	/* Number of members in the following
 									 * arrays */
-	ResultRelInfo *mt_partitions;	/* Per partition result relation */
-	TupleConversionMap **mt_partition_tupconv_maps;
+	ResultRelInfo **mt_partitions;	/* Per partition result relation pointers */
+
 	/* Per partition tuple conversion map */
+	TupleConversionMap **mt_partition_tupconv_maps;
+	/* Per resultRelInfo conversion map to convert tuples to root partition */
+	TupleConversionMap **mt_resultrel_maps;
+
 	TupleTableSlot *mt_partition_tuple_slot;
 	struct TransitionCaptureState *mt_transition_capture;
 									/* controls transition table population */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f1a1b24..cd670b9 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -219,6 +219,7 @@ typedef struct ModifyTable
 	Index		nominalRelation;	/* Parent RT index for use of EXPLAIN */
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
+	bool		part_cols_updated;	/* some part col in hierarchy updated */
 	List	   *resultRelations;	/* integer list of RT indexes */
 	int			resultRelIndex; /* index of first resultRel in plan's list */
 	int			rootResultRelIndex; /* index of the partitioned table root */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9bae3c6..3013964 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1579,6 +1579,7 @@ typedef struct ModifyTablePath
 	Index		nominalRelation;	/* Parent RT index for use of EXPLAIN */
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
+	bool		part_cols_updated;	/* some part col in hierarchy updated */
 	List	   *resultRelations;	/* integer list of RT indexes */
 	List	   *subpaths;		/* Path(s) producing source data */
 	List	   *subroots;		/* per-target-table PlannerInfos */
@@ -2019,6 +2020,10 @@ typedef struct AppendRelInfo
  * The child_rels list must contain at least one element, because the parent
  * partitioned table is itself counted as a child.
  *
+ * all_part_cols contains all attribute numbers from the parent that are
+ * used as partitioning columns by the parent or some descendent which is
+ * itself partitioned.
+ *
  * These structs are kept in the PlannerInfo node's pcinfo_list.
  */
 typedef struct PartitionedChildRelInfo
@@ -2027,6 +2032,7 @@ typedef struct PartitionedChildRelInfo
 
 	Index		parent_relid;
 	List	   *child_rels;
+	Bitmapset  *all_part_cols;
 } PartitionedChildRelInfo;
 
 /*
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 0c0549d..d35f448 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -235,6 +235,7 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						CmdType operation, bool canSetTag,
 						Index nominalRelation, List *partitioned_rels,
+						bool part_cols_updated,
 						List *resultRelations, List *subpaths,
 						List *subroots,
 						List *withCheckOptionLists, List *returningLists,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index 2a4cf71..c6c15c5 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -57,6 +57,7 @@ extern Expr *preprocess_phv_expression(PlannerInfo *root, Expr *expr);
 
 extern bool plan_cluster_use_sort(Oid tableOid, Oid indexOid);
 
-extern List *get_partitioned_child_rels(PlannerInfo *root, Index rti);
+extern List *get_partitioned_child_rels(PlannerInfo *root, Index rti,
+										Bitmapset **all_part_cols_p);
 
 #endif							/* PLANNER_H */
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index 9366f04..20d4878 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -198,25 +198,185 @@ INSERT INTO upsert_test VALUES (1, 'Bat') ON CONFLICT(a)
 
 DROP TABLE update_test;
 DROP TABLE upsert_test;
--- update to a partition should check partition bound constraint for the new tuple
-create table range_parted (
+-- update to a partition should check partition bound constraint for the new tuple.
+-- If partition key is updated, the row should be moved to the appropriate
+-- partition. updatable views using partitions should enforce the check options
+-- for the rows that have been moved.
+create table mintab(c1 int);
+insert into mintab values (120);
+CREATE TABLE range_parted (
 	a text,
-	b int
+	b int,
+	c int
 ) partition by range (a, b);
+CREATE VIEW upview AS SELECT * FROM range_parted WHERE (select c > c1 from mintab) WITH CHECK OPTION;
 create table part_a_1_a_10 partition of range_parted for values from ('a', 1) to ('a', 10);
 create table part_a_10_a_20 partition of range_parted for values from ('a', 10) to ('a', 20);
 create table part_b_1_b_10 partition of range_parted for values from ('b', 1) to ('b', 10);
-create table part_b_10_b_20 partition of range_parted for values from ('b', 10) to ('b', 20);
+create table part_b_10_b_20 partition of range_parted for values from ('b', 10) to ('b', 20) partition by range (c);
+-- This tests partition-key UPDATE on a partitioned table that does not have any child partitions
+update part_b_10_b_20 set b = b - 6;
+create table part_c_1_100 (b int, c int, a text);
+alter table part_b_10_b_20 attach partition part_c_1_100 for values from (1) to (100);
+create table part_c_100_200 (c int, a text, b int);
+alter table part_b_10_b_20 attach partition part_c_100_200 for values from (100) to (200);
 insert into part_a_1_a_10 values ('a', 1);
-insert into part_b_10_b_20 values ('b', 10);
--- fail
-update part_a_1_a_10 set a = 'b' where a = 'a';
-ERROR:  new row for relation "part_a_1_a_10" violates partition constraint
-DETAIL:  Failing row contains (b, 1).
-update range_parted set b = b - 1 where b = 10;
-ERROR:  new row for relation "part_b_10_b_20" violates partition constraint
-DETAIL:  Failing row contains (b, 9).
--- ok
-update range_parted set b = b + 1 where b = 10;
+insert into part_a_10_a_20 values ('a', 10, 200);
+insert into part_c_1_100 (a, b, c) values ('b', 12, 96);
+insert into part_c_1_100 (a, b, c) values ('b', 13, 97);
+insert into part_c_100_200 (a, b, c) values ('b', 15, 105);
+insert into part_c_100_200 (a, b, c) values ('b', 17, 105);
+-- fail (row movement happens only within the partition subtree) :
+update part_c_1_100 set c = c + 20 where c = 96;
+ERROR:  new row for relation "part_c_1_100" violates partition constraint
+DETAIL:  Failing row contains (12, 116, b).
+-- No row found :
+update part_c_1_100 set c = c + 20 where c = 98;
+-- ok (row movement)
+update part_b_10_b_20 set c = c + 20 returning c, b, a;
+  c  | b  | a 
+-----+----+---
+ 116 | 12 | b
+ 117 | 13 | b
+ 125 | 15 | b
+ 125 | 17 | b
+(4 rows)
+
+select a, b, c from part_c_1_100 order by 1, 2, 3;
+ a | b | c 
+---+---+---
+(0 rows)
+
+select a, b, c from part_c_100_200 order by 1, 2, 3;
+ a | b  |  c  
+---+----+-----
+ b | 12 | 116
+ b | 13 | 117
+ b | 15 | 125
+ b | 17 | 125
+(4 rows)
+
+-- fail (row movement happens only within the partition subtree) :
+update part_b_10_b_20 set b = b - 6 where c > 116 returning *;
+ERROR:  new row for relation "part_c_100_200" violates partition constraint
+DETAIL:  Failing row contains (117, b, 7).
+-- ok (row movement, with subset of rows moved into different partition)
+update range_parted set b = b - 6 where c > 116 returning a, b + c;
+ a | ?column? 
+---+----------
+ a |      204
+ b |      124
+ b |      134
+ b |      136
+(4 rows)
+
+select tableoid::regclass partname, * from range_parted order by 1, 2, 3, 4;
+    partname    | a | b  |  c  
+----------------+---+----+-----
+ part_a_1_a_10  | a |  1 |    
+ part_a_1_a_10  | a |  4 | 200
+ part_b_1_b_10  | b |  7 | 117
+ part_b_1_b_10  | b |  9 | 125
+ part_c_100_200 | b | 11 | 125
+ part_c_100_200 | b | 12 | 116
+(6 rows)
+
+-- update partition key using updatable view.
+-- succeeds
+update upview set c = 199 where b = 4;
+-- fail, check option violation
+update upview set c = 120 where b = 4;
+ERROR:  new row violates check option for view "upview"
+DETAIL:  Failing row contains (a, 4, 120).
+-- fail, row movement with check option violation
+update upview set a = 'b', b = 15, c = 120 where b = 4;
+ERROR:  new row violates check option for view "upview"
+DETAIL:  Failing row contains (b, 15, 120).
+-- succeeds, row movement , check option passes
+update upview set a = 'b', b = 15 where b = 4;
+select tableoid::regclass partname, * from range_parted order by 1, 2, 3, 4;
+    partname    | a | b  |  c  
+----------------+---+----+-----
+ part_a_1_a_10  | a |  1 |    
+ part_b_1_b_10  | b |  7 | 117
+ part_b_1_b_10  | b |  9 | 125
+ part_c_100_200 | b | 11 | 125
+ part_c_100_200 | b | 12 | 116
+ part_c_100_200 | b | 15 | 199
+(6 rows)
+
 -- cleanup
-drop table range_parted;
+drop view upview;
+drop table mintab, range_parted;
+--------------
+-- UPDATE with
+-- partition key or non-partition columns, with different column ordering,
+-- triggers.
+--------------
+-- Setup
+--------
+create table list_parted (a int, b int, c int) partition by list (a);
+create table sub_parted partition of list_parted for values in (1) partition by list (b);
+create table sub_part1(b int, c int, a int);
+alter table sub_parted attach partition sub_part1 for values in (1);
+create table sub_part2(b int, c int, a int);
+alter table sub_parted attach partition sub_part2 for values in (2);
+create table list_part1(a int, b int, c int);
+alter table list_parted attach partition list_part1 for values in (2,3);
+insert into list_parted values (2,5,50);
+insert into list_parted values (3,6,60);
+insert into sub_parted values (1,1,60);
+insert into sub_parted values (1,2,10);
+-- Test partition constraint violation when intermediate ancestor is used and
+-- constraint is inherited from upper root.
+update sub_parted set a = 2 where c = 10;
+ERROR:  new row for relation "sub_part2" violates partition constraint
+DETAIL:  Failing row contains (2, 10, 2).
+-- UPDATE which does not modify partition key of partitions that are chosen for update.
+select tableoid::regclass , * from list_parted where a = 2 order by 1;
+  tableoid  | a | b | c  
+------------+---+---+----
+ list_part1 | 2 | 5 | 50
+(1 row)
+
+update list_parted set b = c + a where a = 2;
+select tableoid::regclass , * from list_parted where a = 2 order by 1;
+  tableoid  | a | b  | c  
+------------+---+----+----
+ list_part1 | 2 | 52 | 50
+(1 row)
+
+-----------
+-- Triggers can cause UPDATE row movement if it modified partition key.
+-----------
+create function func_parted_mod_b() returns trigger as $$
+begin
+   NEW.b = 2; -- This is changing partition key column.
+   return NEW;
+end $$ language plpgsql;
+create trigger parted_mod_b before update on sub_part1
+   for each row execute procedure func_parted_mod_b();
+select tableoid::regclass , * from list_parted order by 1, 2, 3, 4;
+  tableoid  | a | b  | c  
+------------+---+----+----
+ sub_part1  | 1 |  1 | 60
+ sub_part2  | 1 |  2 | 10
+ list_part1 | 2 | 52 | 50
+ list_part1 | 3 |  6 | 60
+(4 rows)
+
+-- This should do the tuple routing even though there is no explicit
+-- partition-key update, because there is a trigger on sub_part1
+update list_parted set c = 70 where b  = 1 ;
+select tableoid::regclass , * from list_parted order by 1, 2, 3, 4;
+  tableoid  | a | b  | c  
+------------+---+----+----
+ sub_part2  | 1 |  2 | 10
+ sub_part2  | 1 |  2 | 70
+ list_part1 | 2 | 52 | 50
+ list_part1 | 3 |  6 | 60
+(4 rows)
+
+drop trigger parted_mod_b ON sub_part1 ;
+drop function func_parted_mod_b ( ) ;
+drop table list_parted;
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 6637119..edaf19a 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -107,23 +107,126 @@ INSERT INTO upsert_test VALUES (1, 'Bat') ON CONFLICT(a)
 DROP TABLE update_test;
 DROP TABLE upsert_test;
 
--- update to a partition should check partition bound constraint for the new tuple
-create table range_parted (
+-- update to a partition should check partition bound constraint for the new tuple.
+-- If partition key is updated, the row should be moved to the appropriate
+-- partition. updatable views using partitions should enforce the check options
+-- for the rows that have been moved.
+create table mintab(c1 int);
+insert into mintab values (120);
+CREATE TABLE range_parted (
 	a text,
-	b int
+	b int,
+	c int
 ) partition by range (a, b);
+CREATE VIEW upview AS SELECT * FROM range_parted WHERE (select c > c1 from mintab) WITH CHECK OPTION;
+
 create table part_a_1_a_10 partition of range_parted for values from ('a', 1) to ('a', 10);
 create table part_a_10_a_20 partition of range_parted for values from ('a', 10) to ('a', 20);
 create table part_b_1_b_10 partition of range_parted for values from ('b', 1) to ('b', 10);
-create table part_b_10_b_20 partition of range_parted for values from ('b', 10) to ('b', 20);
-insert into part_a_1_a_10 values ('a', 1);
-insert into part_b_10_b_20 values ('b', 10);
+create table part_b_10_b_20 partition of range_parted for values from ('b', 10) to ('b', 20) partition by range (c);
+
+-- This tests partition-key UPDATE on a partitioned table that does not have any child partitions
+update part_b_10_b_20 set b = b - 6;
 
--- fail
-update part_a_1_a_10 set a = 'b' where a = 'a';
-update range_parted set b = b - 1 where b = 10;
--- ok
-update range_parted set b = b + 1 where b = 10;
+create table part_c_1_100 (b int, c int, a text);
+alter table part_b_10_b_20 attach partition part_c_1_100 for values from (1) to (100);
+create table part_c_100_200 (c int, a text, b int);
+alter table part_b_10_b_20 attach partition part_c_100_200 for values from (100) to (200);
+
+insert into part_a_1_a_10 values ('a', 1);
+insert into part_a_10_a_20 values ('a', 10, 200);
+insert into part_c_1_100 (a, b, c) values ('b', 12, 96);
+insert into part_c_1_100 (a, b, c) values ('b', 13, 97);
+insert into part_c_100_200 (a, b, c) values ('b', 15, 105);
+insert into part_c_100_200 (a, b, c) values ('b', 17, 105);
+
+-- fail (row movement happens only within the partition subtree) :
+update part_c_1_100 set c = c + 20 where c = 96;
+-- No row found :
+update part_c_1_100 set c = c + 20 where c = 98;
+-- ok (row movement)
+update part_b_10_b_20 set c = c + 20 returning c, b, a;
+select a, b, c from part_c_1_100 order by 1, 2, 3;
+select a, b, c from part_c_100_200 order by 1, 2, 3;
+
+-- fail (row movement happens only within the partition subtree) :
+update part_b_10_b_20 set b = b - 6 where c > 116 returning *;
+-- ok (row movement, with subset of rows moved into different partition)
+update range_parted set b = b - 6 where c > 116 returning a, b + c;
+
+select tableoid::regclass partname, * from range_parted order by 1, 2, 3, 4;
+
+-- update partition key using updatable view.
+
+-- succeeds
+update upview set c = 199 where b = 4;
+-- fail, check option violation
+update upview set c = 120 where b = 4;
+-- fail, row movement with check option violation
+update upview set a = 'b', b = 15, c = 120 where b = 4;
+-- succeeds, row movement , check option passes
+update upview set a = 'b', b = 15 where b = 4;
+
+select tableoid::regclass partname, * from range_parted order by 1, 2, 3, 4;
 
 -- cleanup
-drop table range_parted;
+drop view upview;
+drop table mintab, range_parted;
+
+
+
+--------------
+-- UPDATE with
+-- partition key or non-partition columns, with different column ordering,
+-- triggers.
+--------------
+
+-- Setup
+--------
+create table list_parted (a int, b int, c int) partition by list (a);
+create table sub_parted partition of list_parted for values in (1) partition by list (b);
+
+create table sub_part1(b int, c int, a int);
+alter table sub_parted attach partition sub_part1 for values in (1);
+create table sub_part2(b int, c int, a int);
+alter table sub_parted attach partition sub_part2 for values in (2);
+
+create table list_part1(a int, b int, c int);
+alter table list_parted attach partition list_part1 for values in (2,3);
+
+insert into list_parted values (2,5,50);
+insert into list_parted values (3,6,60);
+insert into sub_parted values (1,1,60);
+insert into sub_parted values (1,2,10);
+
+-- Test partition constraint violation when intermediate ancestor is used and
+-- constraint is inherited from upper root.
+update sub_parted set a = 2 where c = 10;
+
+-- UPDATE which does not modify partition key of partitions that are chosen for update.
+select tableoid::regclass , * from list_parted where a = 2 order by 1;
+update list_parted set b = c + a where a = 2;
+select tableoid::regclass , * from list_parted where a = 2 order by 1;
+
+
+-----------
+-- Triggers can cause UPDATE row movement if it modified partition key.
+-----------
+create function func_parted_mod_b() returns trigger as $$
+begin
+   NEW.b = 2; -- This is changing partition key column.
+   return NEW;
+end $$ language plpgsql;
+create trigger parted_mod_b before update on sub_part1
+   for each row execute procedure func_parted_mod_b();
+
+select tableoid::regclass , * from list_parted order by 1, 2, 3, 4;
+
+-- This should do the tuple routing even though there is no explicit
+-- partition-key update, because there is a trigger on sub_part1
+update list_parted set c = 70 where b  = 1 ;
+select tableoid::regclass , * from list_parted order by 1, 2, 3, 4;
+
+drop trigger parted_mod_b ON sub_part1 ;
+drop function func_parted_mod_b ( ) ;
+drop table list_parted;
