From 8c55a95e538e07d4b74d2985f01684749c791708 Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Thu, 15 Dec 2016 16:27:04 +0900
Subject: [PATCH 2/3] Fix multiple issues with partition constraints

1. A bug in how they are generated

Firstly, since we always want to recurse when calling
RelationGetPartitionQual(), ie, consider the parent's partition
constraint (if any), get rid of the argument recurse; also in
the module-local generate_partition_qual() that it calls.

Move the code for doing parent attnos to child attnos mapping for
Vars in partition constraint expressions to a separate function
map_partition_varattnos() and call it from appropriate places.  Doing
it in get_qual_from_partbound(), as is now, would produce wrong result
in certain multi-level partitioning cases, because it only considers
the current pair of parent-child relations.  In certain multi-level
partitioning cases, attnums for the same key attribute(s) might differ
between multiple pairs of consecutive levels causing the same attribute
to be numbered differently in different Vars of the same expression
tree.  Remember that we apply the whole partition constraint (list of
constraints of partitions at various levels) to a single (leaf
partition) relation.

With this commit, in generate_partition_qual(), we first generate the
the whole partition constraint (considering all levels of partitioning)
and then do the mapping from the root parent attnums to leaf partition
attnums.

2. A bug when attaching a partitioned table as partition

ATExecAttachPartition() failed to do the attnum mapping in the case where
attached partition is a partitioned table.  It is possible in such case
that the leaf partitions of such a table that will be scanned for partition
constraint validation might have different attnums than their parent.
That might lead to incorrect results.

3. A bug when inserting into an internal partition.

Since implicit partition constraints are not inherited, an internal
partition's constraint was not being enforced when targeted directly.
So, include such constraint when setting up leaf partition result
relations for tuple-routing.

InitResultRelInfo()'s API changes with this.  Instead of passing
a boolean telling whether or not to load the partition constraint,
callers now need to pass the exact constraint expression to use
as ri_PartitionCheck or NIL.

4. A bug due to using the wrong TupleTableSlot after tuple routing

We must use the partition's tuple descriptor *after* a tuple is routed,
not the root table's.  Partition's attributes, for example, may be
ordered diferently from the root table's, causing spurious not-null
violation errors due to asking for out-of-range attnum from a heap
tuple and getting null in return.

We must then switch back to the root table's for the next tuple, because
computing partition key of a tuple to be routed must be looking at the
root table's tuple descriptor.  A dedicated TupleTableSlot is allocated
within EState called es_partition_tuple_slot whose descriptor is set to
a given leaf partition for every input tuple after it's routed.

5. A bug whereby ExecConstraints() shows wrong input row in error msgs

After a tuple is routed to a partition, it has been converted from the
root table's rowtype to the partition's.  If such a tuple causes an
error in ExecConstraints(), the row shown in error messages might not
match the input row due to possible differences between the root
table's rowtype and the partition's.

To convert back to the correct row format, keep root table relation
descriptor and a reverse tuple conversion map in the ResultRelInfo's
of leaf partitions.

Reported by: n/a
Patch by: Amit Langote
Reports: n/a
---
 src/backend/catalog/partition.c           | 100 +++++++++++++------------
 src/backend/commands/copy.c               |  35 ++++++++-
 src/backend/commands/tablecmds.c          |  11 ++-
 src/backend/executor/execMain.c           | 120 ++++++++++++++++++++++++++----
 src/backend/executor/nodeModifyTable.c    |  25 +++++++
 src/backend/optimizer/util/plancat.c      |   2 +-
 src/include/catalog/partition.h           |   3 +-
 src/include/executor/executor.h           |   4 +-
 src/include/nodes/execnodes.h             |   5 ++
 src/test/regress/expected/alter_table.out |  26 +++++++
 src/test/regress/expected/insert.out      |  36 +++++++++
 src/test/regress/sql/alter_table.sql      |  20 +++++
 src/test/regress/sql/insert.sql           |  22 ++++++
 13 files changed, 340 insertions(+), 69 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 9980582b77..b6d0841c9b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -122,7 +122,7 @@ static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
 static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
-static List *generate_partition_qual(Relation rel, bool recurse);
+static List *generate_partition_qual(Relation rel);
 
 static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
 					 List *datums, bool lower);
@@ -850,10 +850,6 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
 	PartitionBoundSpec *spec = (PartitionBoundSpec *) bound;
 	PartitionKey key = RelationGetPartitionKey(parent);
 	List	   *my_qual = NIL;
-	TupleDesc	parent_tupdesc = RelationGetDescr(parent);
-	AttrNumber	parent_attno;
-	AttrNumber *partition_attnos;
-	bool		found_whole_row;
 
 	Assert(key != NULL);
 
@@ -874,38 +870,48 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
 				 (int) key->strategy);
 	}
 
-	/*
-	 * Translate vars in the generated expression to have correct attnos. Note
-	 * that the vars in my_qual bear attnos dictated by key which carries
-	 * physical attnos of the parent.  We must allow for a case where physical
-	 * attnos of a partition can be different from the parent.
-	 */
-	partition_attnos = (AttrNumber *)
-		palloc0(parent_tupdesc->natts * sizeof(AttrNumber));
-	for (parent_attno = 1; parent_attno <= parent_tupdesc->natts;
-		 parent_attno++)
+	return my_qual;
+}
+
+/*
+ * map_partition_varattnos - maps varattno of any Vars in expr from the
+ * parent attno to partition attno.
+ *
+ * We must allow for a case where physical attnos of a partition can be
+ * different from the parent's.
+ */
+List *
+map_partition_varattnos(List *expr, Relation partrel, Relation parent)
+{
+	TupleDesc	tupdesc = RelationGetDescr(parent);
+	AttrNumber	attno;
+	AttrNumber *part_attnos;
+	bool		found_whole_row;
+
+	part_attnos = (AttrNumber *) palloc0(tupdesc->natts * sizeof(AttrNumber));
+	for (attno = 1; attno <= tupdesc->natts; attno++)
 	{
-		Form_pg_attribute attribute = parent_tupdesc->attrs[parent_attno - 1];
+		Form_pg_attribute attribute = tupdesc->attrs[attno - 1];
 		char	   *attname = NameStr(attribute->attname);
-		AttrNumber	partition_attno;
+		AttrNumber	part_attno;
 
 		if (attribute->attisdropped)
 			continue;
 
-		partition_attno = get_attnum(RelationGetRelid(rel), attname);
-		partition_attnos[parent_attno - 1] = partition_attno;
+		part_attno = get_attnum(RelationGetRelid(partrel), attname);
+		part_attnos[attno - 1] = part_attno;
 	}
 
-	my_qual = (List *) map_variable_attnos((Node *) my_qual,
-										   1, 0,
-										   partition_attnos,
-										   parent_tupdesc->natts,
-										   &found_whole_row);
-	/* there can never be a whole-row reference here */
+	expr = (List *) map_variable_attnos((Node *) expr,
+										1, 0,
+										part_attnos,
+										tupdesc->natts,
+										&found_whole_row);
+	/* There can never be a whole-row reference here */
 	if (found_whole_row)
 		elog(ERROR, "unexpected whole-row reference found in partition key");
 
-	return my_qual;
+	return expr;
 }
 
 /*
@@ -914,13 +920,13 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
  * Returns a list of partition quals
  */
 List *
-RelationGetPartitionQual(Relation rel, bool recurse)
+RelationGetPartitionQual(Relation rel)
 {
 	/* Quick exit */
 	if (!rel->rd_rel->relispartition)
 		return NIL;
 
-	return generate_partition_qual(rel, recurse);
+	return generate_partition_qual(rel);
 }
 
 /* Turn an array of OIDs with N elements into a list */
@@ -1445,7 +1451,7 @@ get_partition_operator(PartitionKey key, int col, StrategyNumber strategy,
  * into cache memory.
  */
 static List *
-generate_partition_qual(Relation rel, bool recurse)
+generate_partition_qual(Relation rel)
 {
 	HeapTuple	tuple;
 	MemoryContext oldcxt;
@@ -1459,6 +1465,10 @@ generate_partition_qual(Relation rel, bool recurse)
 	/* Guard against stack overflow due to overly deep partition tree */
 	check_stack_depth();
 
+	/* Recursive callers may not have checked themselves */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
 	/* Grab at least an AccessShareLock on the parent table */
 	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
 					   AccessShareLock);
@@ -1466,20 +1476,18 @@ generate_partition_qual(Relation rel, bool recurse)
 	/* Quick copy */
 	if (rel->rd_partcheck)
 	{
-		if (parent->rd_rel->relispartition && recurse)
-			result = list_concat(generate_partition_qual(parent, true),
-								 copyObject(rel->rd_partcheck));
-		else
-			result = copyObject(rel->rd_partcheck);
+		result = list_concat(generate_partition_qual(parent),
+							 copyObject(rel->rd_partcheck));
 
-		heap_close(parent, AccessShareLock);
+		/* Mark Vars with correct attnos */
+		result = map_partition_varattnos(result, rel, parent);
+
+		/* Keep the parent locked until commit */
+		heap_close(parent, NoLock);
 		return result;
 	}
 
 	/* Get pg_class.relpartbound */
-	if (!rel->rd_rel->relispartition)	/* should not happen */
-		elog(ERROR, "relation \"%s\" has relispartition = false",
-			 RelationGetRelationName(rel));
 	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
 	boundDatum = SysCacheGetAttr(RELOID, tuple,
 								 Anum_pg_class_relpartbound,
@@ -1492,18 +1500,16 @@ generate_partition_qual(Relation rel, bool recurse)
 
 	my_qual = get_qual_from_partbound(rel, parent, bound);
 
-	/* If requested, add parent's quals to the list (if any) */
-	if (parent->rd_rel->relispartition && recurse)
-	{
-		List	   *parent_check;
-
-		parent_check = generate_partition_qual(parent, true);
-		result = list_concat(parent_check, my_qual);
-	}
+	/* Add the parent's quals to the list (if any) */
+	if (parent->rd_rel->relispartition)
+		result = list_concat(generate_partition_qual(parent), my_qual);
 	else
 		result = my_qual;
 
-	/* Save a copy of my_qual in the relcache */
+	/* Mark Vars with correct attnos */
+	result = map_partition_varattnos(result, rel, parent);
+
+	/* Save a copy of *only* this rel's partition qual in the relcache */
 	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
 	rel->rd_partcheck = copyObject(my_qual);
 	MemoryContextSwitchTo(oldcxt);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index d5901651db..9cd84e80c7 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -2294,6 +2294,7 @@ CopyFrom(CopyState cstate)
 	uint64		processed = 0;
 	bool		useHeapMultiInsert;
 	int			nBufferedTuples = 0;
+	List	   *partcheck = NIL;
 
 #define MAX_BUFFERED_TUPLES 1000
 	HeapTuple  *bufferedTuples = NULL;	/* initialize to silence warning */
@@ -2410,6 +2411,10 @@ CopyFrom(CopyState cstate)
 		hi_options |= HEAP_INSERT_FROZEN;
 	}
 
+	/* Don't forget the partition constraints */
+	if (cstate->rel->rd_rel->relispartition)
+		partcheck = RelationGetPartitionQual(cstate->rel);
+
 	/*
 	 * We need a ResultRelInfo so we can use the regular executor's
 	 * index-entry-making machinery.  (There used to be a huge amount of code
@@ -2419,7 +2424,7 @@ CopyFrom(CopyState cstate)
 	InitResultRelInfo(resultRelInfo,
 					  cstate->rel,
 					  1,		/* dummy rangetable index */
-					  true,		/* do load partition check expression */
+					  partcheck, NULL, NULL,
 					  0);
 
 	ExecOpenIndices(resultRelInfo, false);
@@ -2436,6 +2441,15 @@ CopyFrom(CopyState cstate)
 	estate->es_trig_tuple_slot = ExecInitExtraTupleSlot(estate);
 
 	/*
+	 * Initialize a dedicated slot to manipulate tuples of any given
+	 * partition's rowtype.
+	 */
+	if (cstate->partition_dispatch_info)
+		estate->es_partition_tuple_slot = ExecInitExtraTupleSlot(estate);
+	else
+		estate->es_partition_tuple_slot = NULL;
+
+	/*
 	 * It's more efficient to prepare a bunch of tuples for insertion, and
 	 * insert them in one heap_multi_insert() call, than call heap_insert()
 	 * separately for every tuple. However, we can't do that if there are
@@ -2484,7 +2498,8 @@ CopyFrom(CopyState cstate)
 
 	for (;;)
 	{
-		TupleTableSlot *slot;
+		TupleTableSlot *slot,
+					   *oldslot = NULL;
 		bool		skip_tuple;
 		Oid			loaded_oid = InvalidOid;
 
@@ -2571,7 +2586,19 @@ CopyFrom(CopyState cstate)
 			map = cstate->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.  Use a dedicated slot from this point on until
+				 * we're finished dealing with the partition.
+				 */
+				oldslot = slot;
+				slot = estate->es_partition_tuple_slot;
+				Assert(slot != NULL);
+				ExecSetSlotDescriptor(slot, RelationGetDescr(partrel));
 				ExecStoreTuple(tuple, slot, InvalidBuffer, true);
 			}
 
@@ -2667,6 +2694,10 @@ CopyFrom(CopyState cstate)
 			{
 				resultRelInfo = saved_resultRelInfo;
 				estate->es_result_relation_info = resultRelInfo;
+
+				/* Switch back to the slot corresponding to the root table */
+				Assert(oldslot != NULL);
+				slot = oldslot;
 			}
 		}
 	}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 1c219b03dd..67ff1715ea 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -1323,7 +1323,7 @@ ExecuteTruncate(TruncateStmt *stmt)
 		InitResultRelInfo(resultRelInfo,
 						  rel,
 						  0,	/* dummy rangetable index */
-						  false,
+						  NIL, NULL, NULL,
 						  0);
 		resultRelInfo++;
 	}
@@ -13151,7 +13151,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
 	 */
 	partConstraint = list_concat(get_qual_from_partbound(attachRel, rel,
 														 cmd->bound),
-								 RelationGetPartitionQual(rel, true));
+								 RelationGetPartitionQual(rel));
 	partConstraint = (List *) eval_const_expressions(NULL,
 													 (Node *) partConstraint);
 	partConstraint = (List *) canonicalize_qual((Expr *) partConstraint);
@@ -13323,6 +13323,7 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
 			Oid			part_relid = lfirst_oid(lc);
 			Relation	part_rel;
 			Expr	   *constr;
+			List	   *my_constr;
 
 			/* Lock already taken */
 			if (part_relid != RelationGetRelid(attachRel))
@@ -13345,8 +13346,10 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
 			tab = ATGetQueueEntry(wqueue, part_rel);
 
 			constr = linitial(partConstraint);
-			tab->partition_constraint = make_ands_implicit((Expr *) constr);
-
+			my_constr = make_ands_implicit((Expr *) constr);
+			tab->partition_constraint = map_partition_varattnos(my_constr,
+																part_rel,
+																rel);
 			/* keep our lock until commit */
 			if (part_rel != attachRel)
 				heap_close(part_rel, NoLock);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index bca34a509c..055efdcf9c 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -821,13 +821,19 @@ InitPlan(QueryDesc *queryDesc, int eflags)
 			Index		resultRelationIndex = lfirst_int(l);
 			Oid			resultRelationOid;
 			Relation	resultRelation;
+			List	   *partcheck = NIL;
 
 			resultRelationOid = getrelid(resultRelationIndex, rangeTable);
 			resultRelation = heap_open(resultRelationOid, RowExclusiveLock);
+
+			/* Don't forget the partition constraint */
+			if (resultRelation->rd_rel->relispartition)
+				partcheck = RelationGetPartitionQual(resultRelation);
+
 			InitResultRelInfo(resultRelInfo,
 							  resultRelation,
 							  resultRelationIndex,
-							  true,
+							  partcheck, NULL, NULL,
 							  estate->es_instrument);
 			resultRelInfo++;
 		}
@@ -1217,7 +1223,9 @@ void
 InitResultRelInfo(ResultRelInfo *resultRelInfo,
 				  Relation resultRelationDesc,
 				  Index resultRelationIndex,
-				  bool load_partition_check,
+				  List *partition_check,
+				  Relation partition_root,
+				  TupleConversionMap *partition_reverse_map,
 				  int instrument_options)
 {
 	MemSet(resultRelInfo, 0, sizeof(ResultRelInfo));
@@ -1255,10 +1263,13 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo,
 	resultRelInfo->ri_ConstraintExprs = NULL;
 	resultRelInfo->ri_junkFilter = NULL;
 	resultRelInfo->ri_projectReturning = NULL;
-	if (load_partition_check)
-		resultRelInfo->ri_PartitionCheck =
-							RelationGetPartitionQual(resultRelationDesc,
-													 true);
+	resultRelInfo->ri_PartitionCheck = partition_check;
+	/*
+	 * Following fields are only looked at in some tuple-routing cases.
+	 * In other case, they are set to NULL.
+	 */
+	resultRelInfo->ri_PartitionRoot = partition_root;
+	resultRelInfo->ri_PartitionReverseMap = partition_reverse_map;
 }
 
 /*
@@ -1285,6 +1296,7 @@ ExecGetTriggerResultRel(EState *estate, Oid relid)
 	ListCell   *l;
 	Relation	rel;
 	MemoryContext oldcontext;
+	List	   *partcheck = NIL;
 
 	/* First, search through the query result relations */
 	rInfo = estate->es_result_relations;
@@ -1313,6 +1325,10 @@ ExecGetTriggerResultRel(EState *estate, Oid relid)
 	 */
 	rel = heap_open(relid, NoLock);
 
+	/* Don't forget the partition constraint */
+	if (rel->rd_rel->relispartition)
+		partcheck = RelationGetPartitionQual(rel);
+
 	/*
 	 * Make the new entry in the right context.
 	 */
@@ -1321,7 +1337,7 @@ ExecGetTriggerResultRel(EState *estate, Oid relid)
 	InitResultRelInfo(rInfo,
 					  rel,
 					  0,		/* dummy rangetable index */
-					  true,
+					  partcheck, NULL, NULL,
 					  estate->es_instrument);
 	estate->es_trig_target_relations =
 		lappend(estate->es_trig_target_relations, rInfo);
@@ -1767,6 +1783,26 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 				slot_attisnull(slot, attrChk))
 			{
 				char	   *val_desc;
+				Relation	orig_rel = rel;
+				TupleDesc	orig_tupdesc = tupdesc;
+
+				/*
+				 * In case where the tuple is routed, it's been converted
+				 * to the partition's rowtype, which might differ from the
+				 * root table's.  We must convert it back to the root table's
+				 * type so that it's shown correctly in the error message.
+				 */
+				if (resultRelInfo->ri_PartitionRoot)
+				{
+					HeapTuple	tuple = ExecFetchSlotTuple(slot);
+
+					rel = resultRelInfo->ri_PartitionRoot;
+					tupdesc = RelationGetDescr(rel);
+					Assert(resultRelInfo->ri_PartitionReverseMap != NULL);
+					tuple = do_convert_tuple(tuple,
+									resultRelInfo->ri_PartitionReverseMap);
+					ExecStoreTuple(tuple, slot, InvalidBuffer, false);
+				}
 
 				insertedCols = GetInsertedColumns(resultRelInfo, estate);
 				updatedCols = GetUpdatedColumns(resultRelInfo, estate);
@@ -1780,9 +1816,9 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 				ereport(ERROR,
 						(errcode(ERRCODE_NOT_NULL_VIOLATION),
 						 errmsg("null value in column \"%s\" violates not-null constraint",
-							  NameStr(tupdesc->attrs[attrChk - 1]->attname)),
+						  NameStr(orig_tupdesc->attrs[attrChk - 1]->attname)),
 						 val_desc ? errdetail("Failing row contains %s.", val_desc) : 0,
-						 errtablecol(rel, attrChk)));
+						 errtablecol(orig_rel, attrChk)));
 			}
 		}
 	}
@@ -1794,6 +1830,20 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 		if ((failed = ExecRelCheck(resultRelInfo, slot, estate)) != NULL)
 		{
 			char	   *val_desc;
+			Relation	orig_rel = rel;
+
+			/* See the comment above. */
+			if (resultRelInfo->ri_PartitionRoot)
+			{
+				HeapTuple	tuple = ExecFetchSlotTuple(slot);
+
+				rel = resultRelInfo->ri_PartitionRoot;
+				tupdesc = RelationGetDescr(rel);
+				Assert(resultRelInfo->ri_PartitionReverseMap != NULL);
+				tuple = do_convert_tuple(tuple,
+									resultRelInfo->ri_PartitionReverseMap);
+				ExecStoreTuple(tuple, slot, InvalidBuffer, false);
+			}
 
 			insertedCols = GetInsertedColumns(resultRelInfo, estate);
 			updatedCols = GetUpdatedColumns(resultRelInfo, estate);
@@ -1806,9 +1856,9 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 			ereport(ERROR,
 					(errcode(ERRCODE_CHECK_VIOLATION),
 					 errmsg("new row for relation \"%s\" violates check constraint \"%s\"",
-							RelationGetRelationName(rel), failed),
+							RelationGetRelationName(orig_rel), failed),
 			  val_desc ? errdetail("Failing row contains %s.", val_desc) : 0,
-					 errtableconstraint(rel, failed)));
+					 errtableconstraint(orig_rel, failed)));
 		}
 	}
 
@@ -1816,6 +1866,20 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 		!ExecPartitionCheck(resultRelInfo, slot, estate))
 	{
 		char	   *val_desc;
+		Relation	orig_rel = rel;
+
+		/* See the comment above. */
+		if (resultRelInfo->ri_PartitionRoot)
+		{
+			HeapTuple	tuple = ExecFetchSlotTuple(slot);
+
+			rel = resultRelInfo->ri_PartitionRoot;
+			tupdesc = RelationGetDescr(rel);
+			Assert(resultRelInfo->ri_PartitionReverseMap != NULL);
+			tuple = do_convert_tuple(tuple,
+									 resultRelInfo->ri_PartitionReverseMap);
+			ExecStoreTuple(tuple, slot, InvalidBuffer, false);
+		}
 
 		insertedCols = GetInsertedColumns(resultRelInfo, estate);
 		updatedCols = GetUpdatedColumns(resultRelInfo, estate);
@@ -1828,7 +1892,7 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
 		ereport(ERROR,
 				(errcode(ERRCODE_CHECK_VIOLATION),
 				 errmsg("new row for relation \"%s\" violates partition constraint",
-						RelationGetRelationName(rel)),
+						RelationGetRelationName(orig_rel)),
 		  val_desc ? errdetail("Failing row contains %s.", val_desc) : 0));
 	}
 }
@@ -3033,6 +3097,7 @@ ExecSetupPartitionTupleRouting(Relation rel,
 	ListCell   *cell;
 	int			i;
 	ResultRelInfo *leaf_part_rri;
+	List		  *partcheck = NIL;
 
 	/* Get the tuple-routing information and lock partitions */
 	*pd = RelationGetPartitionDispatchInfo(rel, RowExclusiveLock, num_parted,
@@ -3043,12 +3108,23 @@ ExecSetupPartitionTupleRouting(Relation rel,
 	*tup_conv_maps = (TupleConversionMap **) palloc0(*num_partitions *
 										   sizeof(TupleConversionMap *));
 
+	/*
+	 * If the main target rel is a partition, ExecConstraints() as applied to
+	 * each leaf partition must consider its partition constraint, because
+	 * unlike explicit constraints, an implicit partition constraint is not
+	 * inherited.
+	 */
+	if (rel->rd_rel->relispartition)
+		partcheck = RelationGetPartitionQual(rel);
+
 	leaf_part_rri = *partitions;
 	i = 0;
 	foreach(cell, leaf_parts)
 	{
 		Relation	partrel;
 		TupleDesc	part_tupdesc;
+		List	   *my_check = NIL;
+		TupleConversionMap	*reverse_map;
 
 		/*
 		 * We locked all the partitions above including the leaf partitions.
@@ -3070,10 +3146,28 @@ ExecSetupPartitionTupleRouting(Relation rel,
 		(*tup_conv_maps)[i] = convert_tuples_by_name(tupDesc, part_tupdesc,
 								 gettext_noop("could not convert row type"));
 
+		/*
+		 * This is the parent's partition constraint, so any Vars in
+		 * it bear the its attribute numbers.  We must switch them to
+		 * the leaf partition's.
+		 */
+		if (partcheck)
+			my_check = map_partition_varattnos(partcheck, partrel, rel);
+
+		/*
+		 * We must save a reverse tuple conversion map as well, to show the
+		 * correct input tuple in the error message shown by ExecConstraints()
+		 * in case of routed tuples.  Remember that at the point of failure,
+		 * the tuple has been converted to the partition's type which might
+		 * not match the input tuple.
+		 */
+		reverse_map = convert_tuples_by_name(part_tupdesc, tupDesc,
+								 gettext_noop("could not convert row type"));
+
 		InitResultRelInfo(leaf_part_rri,
 						  partrel,
 						  1,	 /* dummy */
-						  false,
+						  my_check, rel, reverse_map,
 						  0);
 
 		/*
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index a9546106ce..da4c96a863 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -262,6 +262,7 @@ ExecInsert(ModifyTableState *mtstate,
 	Relation	resultRelationDesc;
 	Oid			newId;
 	List	   *recheckIndexes = NIL;
+	TupleTableSlot *oldslot = NULL;
 
 	/*
 	 * get the heap tuple out of the tuple table slot, making sure we have a
@@ -318,7 +319,19 @@ ExecInsert(ModifyTableState *mtstate,
 		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.
+			 */
+			oldslot = slot;
+			slot = estate->es_partition_tuple_slot;
+			Assert(slot != NULL);
+			ExecSetSlotDescriptor(slot, RelationGetDescr(partrel));
 			ExecStoreTuple(tuple, slot, InvalidBuffer, true);
 		}
 	}
@@ -566,6 +579,10 @@ ExecInsert(ModifyTableState *mtstate,
 	{
 		resultRelInfo = saved_resultRelInfo;
 		estate->es_result_relation_info = resultRelInfo;
+
+		/* Switch back to the slot corresponding to the root table */
+		Assert(oldslot != NULL);
+		slot = oldslot;
 	}
 
 	/*
@@ -1734,7 +1751,15 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		mtstate->mt_partitions = partitions;
 		mtstate->mt_num_partitions = num_partitions;
 		mtstate->mt_partition_tupconv_maps = partition_tupconv_maps;
+
+		/*
+		 * Initialize a dedicated slot to manipulate tuples of any given
+		 * partition's rowtype.
+		 */
+		estate->es_partition_tuple_slot = ExecInitExtraTupleSlot(estate);
 	}
+	else
+		estate->es_partition_tuple_slot = NULL;
 
 	/*
 	 * Initialize any WITH CHECK OPTION constraints if needed.
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 72272d9bb7..150229ed6d 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1228,7 +1228,7 @@ get_relation_constraints(PlannerInfo *root,
 	}
 
 	/* Append partition predicates, if any */
-	pcqual = RelationGetPartitionQual(relation, true);
+	pcqual = RelationGetPartitionQual(relation);
 	if (pcqual)
 	{
 		/*
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 21effbf87b..6ff821e6cf 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -70,7 +70,8 @@ extern bool partition_bounds_equal(PartitionKey key,
 extern void check_new_partition_bound(char *relname, Relation parent, Node *bound);
 extern Oid get_partition_parent(Oid relid);
 extern List *get_qual_from_partbound(Relation rel, Relation parent, Node *bound);
-extern List *RelationGetPartitionQual(Relation rel, bool recurse);
+extern List *map_partition_varattnos(List *expr, Relation partrel, Relation parent);
+extern List *RelationGetPartitionQual(Relation rel);
 
 /* For tuple routing */
 extern PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index b74fa5eb5d..1cc5f2eb94 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -189,7 +189,9 @@ extern void CheckValidResultRel(Relation resultRel, CmdType operation);
 extern void InitResultRelInfo(ResultRelInfo *resultRelInfo,
 				  Relation resultRelationDesc,
 				  Index resultRelationIndex,
-				  bool load_partition_check,
+				  List *partition_check,
+				  Relation partition_root,
+				  TupleConversionMap *partition_reverse_map,
 				  int instrument_options);
 extern ResultRelInfo *ExecGetTriggerResultRel(EState *estate, Oid relid);
 extern bool ExecContextForcesOids(PlanState *planstate, bool *hasoids);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 5c3b8683f5..49db19cc75 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -349,6 +349,8 @@ typedef struct ResultRelInfo
 	List	   *ri_onConflictSetWhere;
 	List	   *ri_PartitionCheck;
 	List	   *ri_PartitionCheckExpr;
+	Relation	ri_PartitionRoot;
+	TupleConversionMap *ri_PartitionReverseMap;
 } ResultRelInfo;
 
 /* ----------------
@@ -384,6 +386,9 @@ typedef struct EState
 	TupleTableSlot *es_trig_oldtup_slot;		/* for TriggerEnabled */
 	TupleTableSlot *es_trig_newtup_slot;		/* for TriggerEnabled */
 
+	/* Slot used to manipulate a tuple after it is routed to a partition */
+	TupleTableSlot *es_partition_tuple_slot;
+
 	/* Parameter info: */
 	ParamListInfo es_param_list_info;	/* values of external params */
 	ParamExecData *es_param_exec_vals;	/* values of internal params */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 99e20eb922..0bf488884a 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3317,3 +3317,29 @@ drop cascades to table part_2
 drop cascades to table part_5
 drop cascades to table part_5_a
 drop cascades to table part_1
+-- multi-level partitions with varying attnums for the same key attribute
+create table p (a int, b int) partition by range (a, b);
+create table p1 (b int, a int not null) partition by range (b);
+create table p11 (like p1);
+alter table p11 drop a;
+alter table p11 add a int;
+alter table p11 drop a;
+alter table p11 add a int not null;
+alter table p1 attach partition p11 for values from (1) to (5);
+-- check attnums for key attribute 'a'
+select attrelid::regclass, attname, attnum
+from pg_attribute
+where attname = 'a' and (attrelid = 'p'::regclass or attrelid = 'p1'::regclass or attrelid = 'p11'::regclass);
+ attrelid | attname | attnum 
+----------+---------+--------
+ p        | a       |      1
+ p1       | a       |      2
+ p11      | a       |      4
+(3 rows)
+
+-- check that attaching partition fails with violating rows in a leaf partition
+insert into p11 (a, b) values (10, 4);
+alter table p attach partition p1 for values from (1, 1) to (1, 10);
+ERROR:  partition constraint is violated by some row
+-- cleanup
+drop table p cascade;
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index 561cefa3c4..e785885723 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -300,3 +300,39 @@ drop cascades to table part_null
 drop cascades to table part_ee_ff
 drop cascades to table part_ee_ff1
 drop cascades to table part_ee_ff2
+-- multi-level partitions with varying attnums for the same key attribute
+create table p (a int, b int) partition by range (a, b);
+create table p1 (b int, a int not null) partition by range (b);
+create table p11 (like p1);
+alter table p11 drop a;
+alter table p11 add a int;
+alter table p11 drop a;
+alter table p11 add a int not null;
+alter table p1 attach partition p11 for values from (1) to (5);
+alter table p attach partition p1 for values from (1, 1) to (1, 10);
+-- check attnums for key attribute 'a'
+select attrelid::regclass, attname, attnum
+from pg_attribute
+where attname = 'a' and (attrelid = 'p'::regclass or attrelid = 'p1'::regclass or attrelid = 'p11'::regclass);
+ attrelid | attname | attnum 
+----------+---------+--------
+ p        | a       |      1
+ p1       | a       |      2
+ p11      | a       |      4
+(3 rows)
+
+-- check that following inserts fail due to either partition constraint or tuple-routing failure
+insert into p11 (a, b) values (10, 4);
+ERROR:  new row for relation "p11" violates partition constraint
+DETAIL:  Failing row contains (4, 10).
+insert into p1 (a, b) values (10, 4);
+ERROR:  new row for relation "p11" violates partition constraint
+DETAIL:  Failing row contains (4, 10).
+insert into p (a, b) values (10, 4);
+ERROR:  no partition of relation "p" found for row
+DETAIL:  Failing row contains (10, 4).
+-- cleanup
+drop table p cascade;
+NOTICE:  drop cascades to 2 other objects
+DETAIL:  drop cascades to table p1
+drop cascades to table p11
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index b285a406d9..2cd9d16648 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2169,3 +2169,23 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 
 -- cleanup
 DROP TABLE list_parted, list_parted2, range_parted CASCADE;
+
+-- multi-level partitions with varying attnums for the same key attribute
+create table p (a int, b int) partition by range (a, b);
+create table p1 (b int, a int not null) partition by range (b);
+create table p11 (like p1);
+alter table p11 drop a;
+alter table p11 add a int;
+alter table p11 drop a;
+alter table p11 add a int not null;
+alter table p1 attach partition p11 for values from (1) to (5);
+-- check attnums for key attribute 'a'
+select attrelid::regclass, attname, attnum
+from pg_attribute
+where attname = 'a' and (attrelid = 'p'::regclass or attrelid = 'p1'::regclass or attrelid = 'p11'::regclass);
+-- check that attaching partition fails with violating rows in a leaf partition
+insert into p11 (a, b) values (10, 4);
+alter table p attach partition p1 for values from (1, 1) to (1, 10);
+
+-- cleanup
+drop table p cascade;
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index 846bb5897a..656bae2d57 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -170,3 +170,25 @@ select tableoid::regclass, * from list_parted;
 -- cleanup
 drop table range_parted cascade;
 drop table list_parted cascade;
+
+-- multi-level partitions with varying attnums for the same key attribute
+create table p (a int, b int) partition by range (a, b);
+create table p1 (b int, a int not null) partition by range (b);
+create table p11 (like p1);
+alter table p11 drop a;
+alter table p11 add a int;
+alter table p11 drop a;
+alter table p11 add a int not null;
+alter table p1 attach partition p11 for values from (1) to (5);
+alter table p attach partition p1 for values from (1, 1) to (1, 10);
+-- check attnums for key attribute 'a'
+select attrelid::regclass, attname, attnum
+from pg_attribute
+where attname = 'a' and (attrelid = 'p'::regclass or attrelid = 'p1'::regclass or attrelid = 'p11'::regclass);
+-- check that following inserts fail due to either partition constraint or tuple-routing failure
+insert into p11 (a, b) values (10, 4);
+insert into p1 (a, b) values (10, 4);
+insert into p (a, b) values (10, 4);
+
+-- cleanup
+drop table p cascade;
-- 
2.11.0

