reorganizing partitioning code (was: Re: [HACKERS] path toward faster partition pruning)

Started by Amit Langotealmost 8 years ago20 messages
#1Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
1 attachment(s)

Re-posting my earlier email to start a new thread.

On 2018/02/09 2:58, Alvaro Herrera wrote:> Robert Haas wrote:

On Wed, Feb 7, 2018 at 3:42 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

partition.c seems to have two kinds of functions 1. that build and
manage relcache, creates quals from bounds etc. which are metadata
management kind 2. partition bound comparison functions, and other
optimizer related functions. May be we should divide the file that
way. The first category code remains in catalog/ as it is today. The
second catagory functions move to optimizer/.

It would be sensible to separate functions that build and manage data
in the relcache from other functions. I think we should consider
moving the existing functions of that type from partition.c to
src/backend/utils/cache/partcache.c.

FWIW I've been thinking that perhaps we need some other separation of
code better than statu quo. The current partition.c file includes stuff
for several modules and ISTM all these new patches are making more and
more of a mess. So +1 to the general idea of splitting things up.
Maybe partcache.c is not ambitious enough, but it seems a good first
step.

Agree with the proposed reorganizing and adding a partcache.c, which I
tried to do in the attached patch.

* The new src/backend/utils/cache/partcache.c contains functions that
initialize relcache's partitioning related fields. Various partition
bound comparison and search functions (and then some) that work off of the
cached information are moved. Also, since we cache partition qual,
interface functions RelationGetPartitioQual(Relation) and
get_partition_qual_relid(Oid) are moved too.

* The new src/include/utils/partcache.h contains various struct
definitions that are moved from backend/catalog/partition.c,
include/catalog/partition.h, and include/utils/rel.h. Also, declarations
of interface functions of partcache.c.

Thoughts?

Thanks,
Amit

Attachments:

v1-0001-Reorganize-partitioning-code.patchtext/plain; charset=UTF-8; name=v1-0001-Reorganize-partitioning-code.patchDownload
From 32020e095b13c48ac5ca7c10cdd75512ab1cf781 Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Tue, 13 Feb 2018 15:59:30 +0900
Subject: [PATCH v1] Reorganize partitioning code

---
 src/backend/catalog/partition.c        | 3906 ++++++++------------------------
 src/backend/executor/execMain.c        |    1 -
 src/backend/executor/execPartition.c   |    1 +
 src/backend/optimizer/prep/prepunion.c |    2 +-
 src/backend/utils/adt/ruleutils.c      |    1 -
 src/backend/utils/cache/Makefile       |    6 +-
 src/backend/utils/cache/partcache.c    | 2114 +++++++++++++++++
 src/backend/utils/cache/relcache.c     |  205 +-
 src/include/catalog/partition.h        |   41 -
 src/include/executor/execPartition.h   |    2 +-
 src/include/utils/partcache.h          |  191 ++
 src/include/utils/rel.h                |   73 +-
 12 files changed, 3301 insertions(+), 3242 deletions(-)
 create mode 100644 src/backend/utils/cache/partcache.c
 create mode 100644 src/include/utils/partcache.h

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 31c80c7f1a..b93768f7c8 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,11 +15,7 @@
 
 #include "postgres.h"
 
-#include "access/hash.h"
-#include "access/heapam.h"
 #include "access/htup_details.h"
-#include "access/nbtree.h"
-#include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
@@ -52,98 +48,9 @@
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
-#include "utils/ruleutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-	char		strategy;		/* hash, list or range? */
-	int			ndatums;		/* Length of the datums following array */
-	Datum	  **datums;
-	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-									 * NULL for hash and list partitioned
-									 * tables */
-	int		   *indexes;		/* Partition indexes */
-	int			null_index;		/* Index of the null-accepting partition; -1
-								 * if there isn't one */
-	int			default_index;	/* Index of the default partition; -1 if there
-								 * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-	int			modulus;
-	int			remainder;
-	int			index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-	int			index;
-	Datum		value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-	int			index;
-	Datum	   *datums;			/* range bound datums */
-	PartitionRangeDatumKind *kind;	/* the kind of each datum */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
-static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
-static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
-							   void *arg);
-static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
-						   void *arg);
-
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
@@ -159,2948 +66,1204 @@ static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 				   bool for_default);
 static List *get_range_nulltest(PartitionKey key);
-static List *generate_partition_qual(Relation rel);
-
-static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
-					 List *datums, bool lower);
-static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
-					 int remainder2);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal);
-static int partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder);
-
-static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int	get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 
 /* SQL-callable function for use in hash partition CHECK constraints */
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 /*
- * RelationBuildPartitionDesc
- *		Form rel's partition descriptor
+ * check_default_allows_bound
  *
- * Not flushed from the cache by RelationClearRelation() unless changed because
- * of addition or removal of partition.
+ * This function checks if there exists a row in the default partition that
+ * would properly belong to the new partition being added.  If it finds one,
+ * it throws an error.
  */
 void
-RelationBuildPartitionDesc(Relation rel)
+check_default_allows_bound(Relation parent, Relation default_rel,
+						   PartitionBoundSpec *new_spec)
 {
-	List	   *inhoids,
-			   *partoids;
-	Oid		   *oids = NULL;
-	List	   *boundspecs = NIL;
-	ListCell   *cell;
-	int			i,
-				nparts;
-	PartitionKey key = RelationGetPartitionKey(rel);
-	PartitionDesc result;
-	MemoryContext oldcxt;
-
-	int			ndatums = 0;
-	int			default_index = -1;
-
-	/* Hash partitioning specific */
-	PartitionHashBound **hbounds = NULL;
-
-	/* List partitioning specific */
-	PartitionListValue **all_values = NULL;
-	int			null_index = -1;
+	List	   *new_part_constraints;
+	List	   *def_part_constraints;
+	List	   *all_parts;
+	ListCell   *lc;
 
-	/* Range partitioning specific */
-	PartitionRangeBound **rbounds = NULL;
+	new_part_constraints = (new_spec->strategy == PARTITION_STRATEGY_LIST)
+		? get_qual_for_list(parent, new_spec)
+		: get_qual_for_range(parent, new_spec, false);
+	def_part_constraints =
+		get_proposed_default_constraint(new_part_constraints);
 
 	/*
-	 * The following could happen in situations where rel has a pg_class entry
-	 * but not the pg_partitioned_table entry yet.
+	 * If the existing constraints on the default partition imply that it will
+	 * not contain any row that would belong to the new partition, we can
+	 * avoid scanning the default partition.
 	 */
-	if (key == NULL)
+	if (PartConstraintImpliedByRelConstraint(default_rel, def_part_constraints))
+	{
+		ereport(INFO,
+				(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
+						RelationGetRelationName(default_rel))));
 		return;
+	}
 
-	/* Get partition oids from pg_inherits */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	/*
+	 * Scan the default partition and its subpartitions, and check for rows
+	 * that do not satisfy the revised partition constraints.
+	 */
+	if (default_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+		all_parts = find_all_inheritors(RelationGetRelid(default_rel),
+										AccessExclusiveLock, NULL);
+	else
+		all_parts = list_make1_oid(RelationGetRelid(default_rel));
 
-	/* Collect bound spec nodes in a list */
-	i = 0;
-	partoids = NIL;
-	foreach(cell, inhoids)
+	foreach(lc, all_parts)
 	{
-		Oid			inhrelid = lfirst_oid(cell);
+		Oid			part_relid = lfirst_oid(lc);
+		Relation	part_rel;
+		Expr	   *constr;
+		Expr	   *partition_constraint;
+		EState	   *estate;
 		HeapTuple	tuple;
-		Datum		datum;
-		bool		isnull;
-		Node	   *boundspec;
-
-		tuple = SearchSysCache1(RELOID, inhrelid);
-		if (!HeapTupleIsValid(tuple))
-			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+		ExprState  *partqualstate = NULL;
+		Snapshot	snapshot;
+		TupleDesc	tupdesc;
+		ExprContext *econtext;
+		HeapScanDesc scan;
+		MemoryContext oldCxt;
+		TupleTableSlot *tupslot;
 
-		/*
-		 * It is possible that the pg_class tuple of a partition has not been
-		 * updated yet to set its relpartbound field.  The only case where
-		 * this happens is when we open the parent relation to check using its
-		 * partition descriptor that a new partition's bound does not overlap
-		 * some existing partition.
-		 */
-		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		/* Lock already taken above. */
+		if (part_relid != RelationGetRelid(default_rel))
 		{
-			ReleaseSysCache(tuple);
-			continue;
-		}
+			part_rel = heap_open(part_relid, NoLock);
+
+			/*
+			 * If the partition constraints on default partition child imply
+			 * that it will not contain any row that would belong to the new
+			 * partition, we can avoid scanning the child table.
+			 */
+			if (PartConstraintImpliedByRelConstraint(part_rel,
+													 def_part_constraints))
+			{
+				ereport(INFO,
+						(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
+								RelationGetRelationName(part_rel))));
 
-		datum = SysCacheGetAttr(RELOID, tuple,
-								Anum_pg_class_relpartbound,
-								&isnull);
-		Assert(!isnull);
-		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+				heap_close(part_rel, NoLock);
+				continue;
+			}
+		}
+		else
+			part_rel = default_rel;
 
 		/*
-		 * Sanity check: If the PartitionBoundSpec says this is the default
-		 * partition, its OID should correspond to whatever's stored in
-		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 * Only RELKIND_RELATION relations (i.e. leaf partitions) need to be
+		 * scanned.
 		 */
-		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		if (part_rel->rd_rel->relkind != RELKIND_RELATION)
 		{
-			Oid			partdefid;
-
-			partdefid = get_default_partition_oid(RelationGetRelid(rel));
-			if (partdefid != inhrelid)
-				elog(ERROR, "expected partdefid %u, but got %u",
-					 inhrelid, partdefid);
-		}
+			if (part_rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+				ereport(WARNING,
+						(errcode(ERRCODE_CHECK_VIOLATION),
+						 errmsg("skipped scanning foreign table \"%s\" which is a partition of default partition \"%s\"",
+								RelationGetRelationName(part_rel),
+								RelationGetRelationName(default_rel))));
 
-		boundspecs = lappend(boundspecs, boundspec);
-		partoids = lappend_oid(partoids, inhrelid);
-		ReleaseSysCache(tuple);
-	}
+			if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
+				heap_close(part_rel, NoLock);
 
-	nparts = list_length(partoids);
+			continue;
+		}
 
-	if (nparts > 0)
-	{
-		oids = (Oid *) palloc(nparts * sizeof(Oid));
-		i = 0;
-		foreach(cell, partoids)
-			oids[i++] = lfirst_oid(cell);
+		tupdesc = CreateTupleDescCopy(RelationGetDescr(part_rel));
+		constr = linitial(def_part_constraints);
+		partition_constraint = (Expr *)
+			map_partition_varattnos((List *) constr,
+									1, part_rel, parent, NULL);
+		estate = CreateExecutorState();
 
-		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_HASH)
-		{
-			ndatums = nparts;
-			hbounds = (PartitionHashBound **)
-				palloc(nparts * sizeof(PartitionHashBound *));
+		/* Build expression execution states for partition check quals */
+		partqualstate = ExecPrepareExpr(partition_constraint, estate);
 
-			i = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
+		econtext = GetPerTupleExprContext(estate);
+		snapshot = RegisterSnapshot(GetLatestSnapshot());
+		scan = heap_beginscan(part_rel, snapshot, 0, NULL);
+		tupslot = MakeSingleTupleTableSlot(tupdesc);
 
-				if (spec->strategy != PARTITION_STRATEGY_HASH)
-					elog(ERROR, "invalid strategy in partition bound spec");
+		/*
+		 * Switch to per-tuple memory context and reset it for each tuple
+		 * produced, so we don't leak memory.
+		 */
+		oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
 
-				hbounds[i] = (PartitionHashBound *)
-					palloc(sizeof(PartitionHashBound));
+		while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
+		{
+			ExecStoreTuple(tuple, tupslot, InvalidBuffer, false);
+			econtext->ecxt_scantuple = tupslot;
 
-				hbounds[i]->modulus = spec->modulus;
-				hbounds[i]->remainder = spec->remainder;
-				hbounds[i]->index = i;
-				i++;
-			}
+			if (!ExecCheck(partqualstate, econtext))
+				ereport(ERROR,
+						(errcode(ERRCODE_CHECK_VIOLATION),
+						 errmsg("updated partition constraint for default partition \"%s\" would be violated by some row",
+								RelationGetRelationName(default_rel))));
 
-			/* Sort all the bounds in ascending order */
-			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
-				  qsort_partition_hbound_cmp);
+			ResetExprContext(econtext);
+			CHECK_FOR_INTERRUPTS();
 		}
-		else if (key->strategy == PARTITION_STRATEGY_LIST)
-		{
-			List	   *non_null_values = NIL;
 
-			/*
-			 * Create a unified list of non-null values across all partitions.
-			 */
-			i = 0;
-			null_index = -1;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				ListCell   *c;
+		MemoryContextSwitchTo(oldCxt);
+		heap_endscan(scan);
+		UnregisterSnapshot(snapshot);
+		ExecDropSingleTupleTableSlot(tupslot);
+		FreeExecutorState(estate);
 
-				if (spec->strategy != PARTITION_STRATEGY_LIST)
-					elog(ERROR, "invalid strategy in partition bound spec");
+		if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
+			heap_close(part_rel, NoLock);	/* keep the lock until commit */
+	}
+}
 
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the list of non-null
-				 * datums for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i;
-					i++;
-					continue;
-				}
+/*
+ * get_partition_parent
+ *
+ * Returns inheritance parent of a partition by scanning pg_inherits
+ *
+ * Note: Because this function assumes that the relation whose OID is passed
+ * as an argument will have precisely one parent, it should only be called
+ * when it is known that the relation is a partition.
+ */
+Oid
+get_partition_parent(Oid relid)
+{
+	Form_pg_inherits form;
+	Relation	catalogRelation;
+	SysScanDesc scan;
+	ScanKeyData key[2];
+	HeapTuple	tuple;
+	Oid			result;
 
-				foreach(c, spec->listdatums)
-				{
-					Const	   *val = castNode(Const, lfirst(c));
-					PartitionListValue *list_value = NULL;
+	catalogRelation = heap_open(InheritsRelationId, AccessShareLock);
 
-					if (!val->constisnull)
-					{
-						list_value = (PartitionListValue *)
-							palloc0(sizeof(PartitionListValue));
-						list_value->index = i;
-						list_value->value = val->constvalue;
-					}
-					else
-					{
-						/*
-						 * Never put a null into the values array, flag
-						 * instead for the code further down below where we
-						 * construct the actual relcache struct.
-						 */
-						if (null_index != -1)
-							elog(ERROR, "found null more than once");
-						null_index = i;
-					}
+	ScanKeyInit(&key[0],
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	ScanKeyInit(&key[1],
+				Anum_pg_inherits_inhseqno,
+				BTEqualStrategyNumber, F_INT4EQ,
+				Int32GetDatum(1));
 
-					if (list_value)
-						non_null_values = lappend(non_null_values,
-												  list_value);
-				}
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId, true,
+							  NULL, 2, key);
 
-				i++;
-			}
+	tuple = systable_getnext(scan);
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
-			ndatums = list_length(non_null_values);
+	form = (Form_pg_inherits) GETSTRUCT(tuple);
+	result = form->inhparent;
 
-			/*
-			 * Collect all list values in one array. Alongside the value, we
-			 * also save the index of partition the value comes from.
-			 */
-			all_values = (PartitionListValue **) palloc(ndatums *
-														sizeof(PartitionListValue *));
-			i = 0;
-			foreach(cell, non_null_values)
-			{
-				PartitionListValue *src = lfirst(cell);
+	systable_endscan(scan);
+	heap_close(catalogRelation, AccessShareLock);
 
-				all_values[i] = (PartitionListValue *)
-					palloc(sizeof(PartitionListValue));
-				all_values[i]->value = src->value;
-				all_values[i]->index = src->index;
-				i++;
-			}
+	return result;
+}
 
-			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
-					  qsort_partition_list_value_cmp, (void *) key);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_RANGE)
-		{
-			int			k;
-			PartitionRangeBound **all_bounds,
-					   *prev;
+/*
+ * get_qual_from_partbound
+ *		Given a parser node for partition bound, return the list of executable
+ *		expressions as partition constraint
+ */
+List *
+get_qual_from_partbound(Relation rel, Relation parent,
+						PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	List	   *my_qual = NIL;
 
-			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
-														  sizeof(PartitionRangeBound *));
+	Assert(key != NULL);
 
-			/*
-			 * Create a unified list of range bounds across all the
-			 * partitions.
-			 */
-			i = ndatums = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				PartitionRangeBound *lower,
-						   *upper;
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+			my_qual = get_qual_for_hash(parent, spec);
+			break;
 
-				if (spec->strategy != PARTITION_STRATEGY_RANGE)
-					elog(ERROR, "invalid strategy in partition bound spec");
+		case PARTITION_STRATEGY_LIST:
+			Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+			my_qual = get_qual_for_list(parent, spec);
+			break;
 
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the allbounds array
-				 * for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i++;
-					continue;
-				}
+		case PARTITION_STRATEGY_RANGE:
+			Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+			my_qual = get_qual_for_range(parent, spec, false);
+			break;
 
-				lower = make_one_range_bound(key, i, spec->lowerdatums,
-											 true);
-				upper = make_one_range_bound(key, i, spec->upperdatums,
-											 false);
-				all_bounds[ndatums++] = lower;
-				all_bounds[ndatums++] = upper;
-				i++;
-			}
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
 
-			Assert(ndatums == nparts * 2 ||
-				   (default_index != -1 && ndatums == (nparts - 1) * 2));
-
-			/* Sort all the bounds in ascending order */
-			qsort_arg(all_bounds, ndatums,
-					  sizeof(PartitionRangeBound *),
-					  qsort_partition_rbound_cmp,
-					  (void *) key);
-
-			/* Save distinct bounds from all_bounds into rbounds. */
-			rbounds = (PartitionRangeBound **)
-				palloc(ndatums * sizeof(PartitionRangeBound *));
-			k = 0;
-			prev = NULL;
-			for (i = 0; i < ndatums; i++)
-			{
-				PartitionRangeBound *cur = all_bounds[i];
-				bool		is_distinct = false;
-				int			j;
+	return my_qual;
+}
 
-				/* Is the current bound distinct from the previous one? */
-				for (j = 0; j < key->partnatts; j++)
-				{
-					Datum		cmpval;
+/*
+ * map_partition_varattnos - maps varattno of any Vars in expr from the
+ * attno's of 'from_rel' to the attno's of 'to_rel' partition, each of which
+ * may be either a leaf partition or a partitioned table, but both of which
+ * must be from the same partitioning hierarchy.
+ *
+ * Even though all of the same column names must be present in all relations
+ * in the hierarchy, and they must also have the same types, the attnos may
+ * be different.
+ *
+ * If found_whole_row is not NULL, *found_whole_row returns whether a
+ * whole-row variable was found in the input expression.
+ *
+ * Note: this will work on any node tree, so really the argument and result
+ * should be declared "Node *".  But a substantial majority of the callers
+ * are working on Lists, so it's less messy to do the casts internally.
+ */
+List *
+map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel,
+						bool *found_whole_row)
+{
+	bool		my_found_whole_row = false;
 
-					if (prev == NULL || cur->kind[j] != prev->kind[j])
-					{
-						is_distinct = true;
-						break;
-					}
+	if (expr != NIL)
+	{
+		AttrNumber *part_attnos;
 
-					/*
-					 * If the bounds are both MINVALUE or MAXVALUE, stop now
-					 * and treat them as equal, since any values after this
-					 * point must be ignored.
-					 */
-					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
-						break;
-
-					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
-											   key->partcollation[j],
-											   cur->datums[j],
-											   prev->datums[j]);
-					if (DatumGetInt32(cmpval) != 0)
-					{
-						is_distinct = true;
-						break;
-					}
-				}
+		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,
+											fromrel_varno, 0,
+											part_attnos,
+											RelationGetDescr(from_rel)->natts,
+											RelationGetForm(to_rel)->reltype,
+											&my_found_whole_row);
+	}
 
-				/*
-				 * Only if the bound is distinct save it into a temporary
-				 * array i.e. rbounds which is later copied into boundinfo
-				 * datums array.
-				 */
-				if (is_distinct)
-					rbounds[k++] = all_bounds[i];
+	if (found_whole_row)
+		*found_whole_row = my_found_whole_row;
 
-				prev = cur;
-			}
+	return expr;
+}
 
-			/* Update ndatums to hold the count of distinct datums. */
-			ndatums = k;
-		}
-		else
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
+/* Module-local functions */
+
+/*
+ * get_partition_operator
+ *
+ * Return oid of the operator of given strategy for a given partition key
+ * column.
+ */
+static Oid
+get_partition_operator(PartitionKey key, int col, StrategyNumber strategy,
+					   bool *need_relabel)
+{
+	Oid			operoid;
 
-	/* Now build the actual relcache partition descriptor */
-	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
-												  RelationGetRelationName(rel),
-												  MEMCONTEXT_COPY_NAME,
-												  ALLOCSET_DEFAULT_SIZES);
-	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+	/*
+	 * First check if there exists an operator of the given strategy, with
+	 * this column's type as both its lefttype and righttype, in the
+	 * partitioning operator family specified for the column.
+	 */
+	operoid = get_opfamily_member(key->partopfamily[col],
+								  key->parttypid[col],
+								  key->parttypid[col],
+								  strategy);
 
-	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
-	result->nparts = nparts;
-	if (nparts > 0)
+	/*
+	 * If one doesn't exist, we must resort to using an operator in the same
+	 * operator family but with the operator class declared input type.  It is
+	 * OK to do so, because the column's type is known to be binary-coercible
+	 * with the operator class input type (otherwise, the operator class in
+	 * question would not have been accepted as the partitioning operator
+	 * class).  We must however inform the caller to wrap the non-Const
+	 * expression with a RelabelType node to denote the implicit coercion. It
+	 * ensures that the resulting expression structurally matches similarly
+	 * processed expressions within the optimizer.
+	 */
+	if (!OidIsValid(operoid))
 	{
-		PartitionBoundInfo boundinfo;
-		int		   *mapping;
-		int			next_index = 0;
-
-		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
-
-		boundinfo = (PartitionBoundInfoData *)
-			palloc0(sizeof(PartitionBoundInfoData));
-		boundinfo->strategy = key->strategy;
-		boundinfo->default_index = -1;
-		boundinfo->ndatums = ndatums;
-		boundinfo->null_index = -1;
-		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
-
-		/* Initialize mapping array with invalid values */
-		mapping = (int *) palloc(sizeof(int) * nparts);
-		for (i = 0; i < nparts; i++)
-			mapping[i] = -1;
+		operoid = get_opfamily_member(key->partopfamily[col],
+									  key->partopcintype[col],
+									  key->partopcintype[col],
+									  strategy);
+		if (!OidIsValid(operoid))
+			elog(ERROR, "missing operator %d(%u,%u) in opfamily %u",
+				 strategy, key->partopcintype[col], key->partopcintype[col],
+				 key->partopfamily[col]);
+		*need_relabel = true;
+	}
+	else
+		*need_relabel = false;
 
-		switch (key->strategy)
-		{
-			case PARTITION_STRATEGY_HASH:
-				{
-					/* Modulus are stored in ascending order */
-					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+	return operoid;
+}
 
-					boundinfo->indexes = (int *) palloc(greatest_modulus *
-														sizeof(int));
+/*
+ * make_partition_op_expr
+ *		Returns an Expr for the given partition key column with arg1 and
+ *		arg2 as its leftop and rightop, respectively
+ */
+static Expr *
+make_partition_op_expr(PartitionKey key, int keynum,
+					   uint16 strategy, Expr *arg1, Expr *arg2)
+{
+	Oid			operoid;
+	bool		need_relabel = false;
+	Expr	   *result = NULL;
 
-					for (i = 0; i < greatest_modulus; i++)
-						boundinfo->indexes[i] = -1;
+	/* Get the correct btree operator for this partitioning column */
+	operoid = get_partition_operator(key, keynum, strategy, &need_relabel);
 
-					for (i = 0; i < nparts; i++)
-					{
-						int			modulus = hbounds[i]->modulus;
-						int			remainder = hbounds[i]->remainder;
-
-						boundinfo->datums[i] = (Datum *) palloc(2 *
-																sizeof(Datum));
-						boundinfo->datums[i][0] = Int32GetDatum(modulus);
-						boundinfo->datums[i][1] = Int32GetDatum(remainder);
-
-						while (remainder < greatest_modulus)
-						{
-							/* overlap? */
-							Assert(boundinfo->indexes[remainder] == -1);
-							boundinfo->indexes[remainder] = i;
-							remainder += modulus;
-						}
-
-						mapping[hbounds[i]->index] = i;
-						pfree(hbounds[i]);
-					}
-					pfree(hbounds);
-					break;
-				}
+	/*
+	 * Chosen operator may be such that the non-Const operand needs to be
+	 * coerced, so apply the same; see the comment in
+	 * get_partition_operator().
+	 */
+	if (!IsA(arg1, Const) &&
+		(need_relabel ||
+		 key->partcollation[keynum] != key->parttypcoll[keynum]))
+		arg1 = (Expr *) makeRelabelType(arg1,
+										key->partopcintype[keynum],
+										-1,
+										key->partcollation[keynum],
+										COERCE_EXPLICIT_CAST);
 
-			case PARTITION_STRATEGY_LIST:
-				{
-					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
-
-					/*
-					 * Copy values.  Indexes of individual values are mapped
-					 * to canonical values so that they match for any two list
-					 * partitioned tables with same number of partitions and
-					 * same lists per partition.  One way to canonicalize is
-					 * to assign the index in all_values[] of the smallest
-					 * value of each partition, as the index of all of the
-					 * partition's values.
-					 */
-					for (i = 0; i < ndatums; i++)
-					{
-						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
-						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
-															key->parttypbyval[0],
-															key->parttyplen[0]);
+	/* Generate the actual expression */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			{
+				List	   *elems = (List *) arg2;
+				int			nelems = list_length(elems);
 
-						/* If the old index has no mapping, assign one */
-						if (mapping[all_values[i]->index] == -1)
-							mapping[all_values[i]->index] = next_index++;
+				Assert(nelems >= 1);
+				Assert(keynum == 0);
 
-						boundinfo->indexes[i] = mapping[all_values[i]->index];
-					}
+				if (nelems > 1 &&
+					!type_is_array(key->parttypid[keynum]))
+				{
+					ArrayExpr  *arrexpr;
+					ScalarArrayOpExpr *saopexpr;
 
-					/*
-					 * If null-accepting partition has no mapped index yet,
-					 * assign one.  This could happen if such partition
-					 * accepts only null and hence not covered in the above
-					 * loop which only handled non-null values.
-					 */
-					if (null_index != -1)
-					{
-						Assert(null_index >= 0);
-						if (mapping[null_index] == -1)
-							mapping[null_index] = next_index++;
-						boundinfo->null_index = mapping[null_index];
-					}
+					/* Construct an ArrayExpr for the right-hand inputs */
+					arrexpr = makeNode(ArrayExpr);
+					arrexpr->array_typeid =
+									get_array_type(key->parttypid[keynum]);
+					arrexpr->array_collid = key->parttypcoll[keynum];
+					arrexpr->element_typeid = key->parttypid[keynum];
+					arrexpr->elements = elems;
+					arrexpr->multidims = false;
+					arrexpr->location = -1;
 
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						/*
-						 * The default partition accepts any value not
-						 * specified in the lists of other partitions, hence
-						 * it should not get mapped index while assigning
-						 * those for non-null datums.
-						 */
-						Assert(default_index >= 0 &&
-							   mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
+					/* Build leftop = ANY (rightop) */
+					saopexpr = makeNode(ScalarArrayOpExpr);
+					saopexpr->opno = operoid;
+					saopexpr->opfuncid = get_opcode(operoid);
+					saopexpr->useOr = true;
+					saopexpr->inputcollid = key->partcollation[keynum];
+					saopexpr->args = list_make2(arg1, arrexpr);
+					saopexpr->location = -1;
 
-					/* All partition must now have a valid mapping */
-					Assert(next_index == nparts);
-					break;
+					result = (Expr *) saopexpr;
 				}
-
-			case PARTITION_STRATEGY_RANGE:
+				else
 				{
-					boundinfo->kind = (PartitionRangeDatumKind **)
-						palloc(ndatums *
-							   sizeof(PartitionRangeDatumKind *));
-					boundinfo->indexes = (int *) palloc((ndatums + 1) *
-														sizeof(int));
+					List	   *elemops = NIL;
+					ListCell   *lc;
 
-					for (i = 0; i < ndatums; i++)
+					foreach (lc, elems)
 					{
-						int			j;
-
-						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
-																sizeof(Datum));
-						boundinfo->kind[i] = (PartitionRangeDatumKind *)
-							palloc(key->partnatts *
-								   sizeof(PartitionRangeDatumKind));
-						for (j = 0; j < key->partnatts; j++)
-						{
-							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
-								boundinfo->datums[i][j] =
-									datumCopy(rbounds[i]->datums[j],
-											  key->parttypbyval[j],
-											  key->parttyplen[j]);
-							boundinfo->kind[i][j] = rbounds[i]->kind[j];
-						}
-
-						/*
-						 * There is no mapping for invalid indexes.
-						 *
-						 * Any lower bounds in the rbounds array have invalid
-						 * indexes assigned, because the values between the
-						 * previous bound (if there is one) and this (lower)
-						 * bound are not part of the range of any existing
-						 * partition.
-						 */
-						if (rbounds[i]->lower)
-							boundinfo->indexes[i] = -1;
-						else
-						{
-							int			orig_index = rbounds[i]->index;
-
-							/* If the old index has no mapping, assign one */
-							if (mapping[orig_index] == -1)
-								mapping[orig_index] = next_index++;
-
-							boundinfo->indexes[i] = mapping[orig_index];
-						}
-					}
+						Expr   *elem = lfirst(lc),
+							   *elemop;
 
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						Assert(default_index >= 0 && mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
+						elemop = make_opclause(operoid,
+											   BOOLOID,
+											   false,
+											   arg1, elem,
+											   InvalidOid,
+											   key->partcollation[keynum]);
+						elemops = lappend(elemops, elemop);
 					}
-					boundinfo->indexes[i] = -1;
-					break;
-				}
 
-			default:
-				elog(ERROR, "unexpected partition strategy: %d",
-					 (int) key->strategy);
-		}
+					result = nelems > 1 ? makeBoolExpr(OR_EXPR, elemops, -1) : linitial(elemops);
+				}
+				break;
+			}
 
-		result->boundinfo = boundinfo;
+		case PARTITION_STRATEGY_RANGE:
+			result = make_opclause(operoid,
+								   BOOLOID,
+								   false,
+								   arg1, arg2,
+								   InvalidOid,
+								   key->partcollation[keynum]);
+			break;
 
-		/*
-		 * Now assign OIDs from the original array into mapped indexes of the
-		 * result array.  Order of OIDs in the former is defined by the
-		 * catalog scan that retrieved them, whereas that in the latter is
-		 * defined by canonicalized representation of the partition bounds.
-		 */
-		for (i = 0; i < nparts; i++)
-			result->oids[mapping[i]] = oids[i];
-		pfree(mapping);
+		default:
+			elog(ERROR, "invalid partitioning strategy");
+			break;
 	}
 
-	MemoryContextSwitchTo(oldcxt);
-	rel->rd_partdesc = result;
+	return result;
 }
 
 /*
- * Are two partition bound collections logically equal?
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder corresponding to a
+ * partition, this function returns CHECK constraint expression Node for that
+ * partition.
  *
- * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
- * This is also useful when b1 and b2 are bound collections of two separate
- * relations, respectively, because PartitionBoundInfo is a canonical
- * representation of partition bounds.
+ * The partition constraint for a hash partition is always a call to the
+ * built-in function satisfies_hash_partition().  The first two arguments are
+ * the modulus and remainder for the partition; the remaining arguments are the
+ * values to be hashed.
  */
-bool
-partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
-					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
 {
+	PartitionKey key = RelationGetPartitionKey(parent);
+	FuncExpr   *fexpr;
+	Node	   *relidConst;
+	Node	   *modulusConst;
+	Node	   *remainderConst;
+	List	   *args;
+	ListCell   *partexprs_item;
 	int			i;
 
-	if (b1->strategy != b2->strategy)
-		return false;
+	/* Fixed arguments. */
+	relidConst = (Node *) makeConst(OIDOID,
+									-1,
+									InvalidOid,
+									sizeof(Oid),
+									ObjectIdGetDatum(RelationGetRelid(parent)),
+									false,
+									true);
 
-	if (b1->ndatums != b2->ndatums)
-		return false;
+	modulusConst = (Node *) makeConst(INT4OID,
+									  -1,
+									  InvalidOid,
+									  sizeof(int32),
+									  Int32GetDatum(spec->modulus),
+									  false,
+									  true);
 
-	if (b1->null_index != b2->null_index)
-		return false;
+	remainderConst = (Node *) makeConst(INT4OID,
+										-1,
+										InvalidOid,
+										sizeof(int32),
+										Int32GetDatum(spec->remainder),
+										false,
+										true);
 
-	if (b1->default_index != b2->default_index)
-		return false;
+	args = list_make3(relidConst, modulusConst, remainderConst);
+	partexprs_item = list_head(key->partexprs);
 
-	if (b1->strategy == PARTITION_STRATEGY_HASH)
-	{
-		int			greatest_modulus = get_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_greatest_modulus(b2))
-			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their modulus
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
-	}
-	else
+	/* Add an argument for each key column. */
+	for (i = 0; i < key->partnatts; i++)
 	{
-		for (i = 0; i < b1->ndatums; i++)
-		{
-			int			j;
-
-			for (j = 0; j < partnatts; j++)
-			{
-				/* For range partitions, the bounds might not be finite. */
-				if (b1->kind != NULL)
-				{
-					/* The different kinds of bound all differ from each other */
-					if (b1->kind[i][j] != b2->kind[i][j])
-						return false;
-
-					/*
-					 * Non-finite bounds are equal without further
-					 * examination.
-					 */
-					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
-						continue;
-				}
-
-				/*
-				 * Compare the actual values. Note that it would be both
-				 * incorrect and unsafe to invoke the comparison operator
-				 * derived from the partitioning specification here.  It would
-				 * be incorrect because we want the relcache entry to be
-				 * updated for ANY change to the partition bounds, not just
-				 * those that the partitioning operator thinks are
-				 * significant.  It would be unsafe because we might reach
-				 * this code in the context of an aborted transaction, and an
-				 * arbitrary partitioning operator might not be safe in that
-				 * context.  datumIsEqual() should be simple enough to be
-				 * safe.
-				 */
-				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-								  parttypbyval[j], parttyplen[j]))
-					return false;
-			}
+		Node	   *keyCol;
 
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
+		/* Left operand */
+		if (key->partattrs[i] != 0)
+		{
+			keyCol = (Node *) makeVar(1,
+									  key->partattrs[i],
+									  key->parttypid[i],
+									  key->parttypmod[i],
+									  key->parttypcoll[i],
+									  0);
+		}
+		else
+		{
+			keyCol = (Node *) copyObject(lfirst(partexprs_item));
+			partexprs_item = lnext(partexprs_item);
 		}
 
-		/* There are ndatums+1 indexes in case of range partitions */
-		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
-			b1->indexes[i] != b2->indexes[i])
-			return false;
+		args = lappend(args, keyCol);
 	}
-	return true;
+
+	fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+						 BOOLOID,
+						 args,
+						 InvalidOid,
+						 InvalidOid,
+						 COERCE_EXPLICIT_CALL);
+
+	return list_make1(fexpr);
 }
 
 /*
- * Return a copy of given PartitionBoundInfo structure. The data types of bounds
- * are described by given partition key specification.
+ * get_qual_for_list
+ *
+ * Returns an implicit-AND list of expressions to use as a list partition's
+ * constraint, given the partition key and bound structures.
+ *
+ * The function returns NIL for a default partition when it's the only
+ * partition since in that case there is no constraint.
  */
-extern PartitionBoundInfo
-partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key)
+static List *
+get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 {
-	PartitionBoundInfo dest;
-	int			i;
-	int			ndatums;
-	int			partnatts;
-	int			num_indexes;
+	PartitionKey key = RelationGetPartitionKey(parent);
+	List	   *result;
+	Expr	   *keyCol;
+	Expr	   *opexpr;
+	NullTest   *nulltest;
+	ListCell   *cell;
+	List	   *elems = NIL;
+	bool		list_has_null = false;
+
+	/*
+	 * Only single-column list partitioning is supported, so we are worried
+	 * only about the partition key with index 0.
+	 */
+	Assert(key->partnatts == 1);
 
-	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	/* Construct Var or expression representing the partition column */
+	if (key->partattrs[0] != 0)
+		keyCol = (Expr *) makeVar(1,
+								  key->partattrs[0],
+								  key->parttypid[0],
+								  key->parttypmod[0],
+								  key->parttypcoll[0],
+								  0);
+	else
+		keyCol = (Expr *) copyObject(linitial(key->partexprs));
 
-	dest->strategy = src->strategy;
-	ndatums = dest->ndatums = src->ndatums;
-	partnatts = key->partnatts;
+	/*
+	 * For default list partition, collect datums for all the partitions. The
+	 * default partition constraint should check that the partition key is
+	 * equal to none of those.
+	 */
+	if (spec->is_default)
+	{
+		int			i;
+		int			ndatums = 0;
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
-	num_indexes = get_partition_bound_num_indexes(src);
+		if (boundinfo)
+		{
+			ndatums = boundinfo->ndatums;
 
-	/* List partitioned tables have only a single partition key. */
-	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+			if (partition_bound_accepts_nulls(boundinfo))
+				list_has_null = true;
+		}
 
-	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+		/*
+		 * If default is the only partition, there need not be any partition
+		 * constraint on it.
+		 */
+		if (ndatums == 0 && !list_has_null)
+			return NIL;
 
-	if (src->kind != NULL)
-	{
-		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
-														 sizeof(PartitionRangeDatumKind *));
 		for (i = 0; i < ndatums; i++)
 		{
-			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
-															   sizeof(PartitionRangeDatumKind));
+			Const	   *val;
+
+			/*
+			 * Construct Const from known-not-null datum.  We must be careful
+			 * to copy the value, because our result has to be able to outlive
+			 * the relcache entry we're copying from.
+			 */
+			val = makeConst(key->parttypid[0],
+							key->parttypmod[0],
+							key->parttypcoll[0],
+							key->parttyplen[0],
+							datumCopy(*boundinfo->datums[i],
+									  key->parttypbyval[0],
+									  key->parttyplen[0]),
+							false,	/* isnull */
+							key->parttypbyval[0]);
 
-			memcpy(dest->kind[i], src->kind[i],
-				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+			elems = lappend(elems, val);
 		}
 	}
 	else
-		dest->kind = NULL;
+	{
+		/*
+		 * Create list of Consts for the allowed values, excluding any nulls.
+		 */
+		foreach(cell, spec->listdatums)
+		{
+			Const	   *val = castNode(Const, lfirst(cell));
+
+			if (val->constisnull)
+				list_has_null = true;
+			else
+				elems = lappend(elems, copyObject(val));
+		}
+	}
 
-	for (i = 0; i < ndatums; i++)
+	if (elems)
 	{
-		int			j;
+		/*
+		 * Generate the operator expression from the non-null partition
+		 * values.
+		 */
+		opexpr = make_partition_op_expr(key, 0, BTEqualStrategyNumber,
+										keyCol, (Expr *) elems);
+	}
+	else
+	{
+		/*
+		 * If there are no partition values, we don't need an operator
+		 * expression.
+		 */
+		opexpr = NULL;
+	}
 
+	if (!list_has_null)
+	{
 		/*
-		 * For a corresponding to hash partition, datums array will have two
-		 * elements - modulus and remainder.
+		 * Gin up a "col IS NOT NULL" test that will be AND'd with the main
+		 * expression.  This might seem redundant, but the partition routing
+		 * machinery needs it.
 		 */
-		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
-		int			natts = hash_part ? 2 : partnatts;
+		nulltest = makeNode(NullTest);
+		nulltest->arg = keyCol;
+		nulltest->nulltesttype = IS_NOT_NULL;
+		nulltest->argisrow = false;
+		nulltest->location = -1;
 
-		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+		result = opexpr ? list_make2(nulltest, opexpr) : list_make1(nulltest);
+	}
+	else
+	{
+		/*
+		 * Gin up a "col IS NULL" test that will be OR'd with the main
+		 * expression.
+		 */
+		nulltest = makeNode(NullTest);
+		nulltest->arg = keyCol;
+		nulltest->nulltesttype = IS_NULL;
+		nulltest->argisrow = false;
+		nulltest->location = -1;
 
-		for (j = 0; j < natts; j++)
+		if (opexpr)
 		{
-			bool		byval;
-			int			typlen;
-
-			if (hash_part)
-			{
-				typlen = sizeof(int32); /* Always int4 */
-				byval = true;	/* int4 is pass-by-value */
-			}
-			else
-			{
-				byval = key->parttypbyval[j];
-				typlen = key->parttyplen[j];
-			}
+			Expr	   *or;
 
-			if (dest->kind == NULL ||
-				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
-				dest->datums[i][j] = datumCopy(src->datums[i][j],
-											   byval, typlen);
+			or = makeBoolExpr(OR_EXPR, list_make2(nulltest, opexpr), -1);
+			result = list_make1(or);
 		}
+		else
+			result = list_make1(nulltest);
 	}
 
-	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
-	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
-
-	dest->null_index = src->null_index;
-	dest->default_index = src->default_index;
+	/*
+	 * Note that, in general, applying NOT to a constraint expression doesn't
+	 * necessarily invert the set of rows it accepts, because NOT (NULL) is
+	 * NULL.  However, the partition constraints we construct here never
+	 * evaluate to NULL, so applying NOT works as intended.
+	 */
+	if (spec->is_default)
+	{
+		result = list_make1(make_ands_explicit(result));
+		result = list_make1(makeBoolExpr(NOT_EXPR, result, -1));
+	}
 
-	return dest;
+	return result;
 }
 
 /*
- * check_new_partition_bound
+ * get_range_key_properties
+ *		Returns range partition key information for a given column
+ *
+ * This is a subroutine for get_qual_for_range, and its API is pretty
+ * specialized to that caller.
+ *
+ * Constructs an Expr for the key column (returned in *keyCol) and Consts
+ * for the lower and upper range limits (returned in *lower_val and
+ * *upper_val).  For MINVALUE/MAXVALUE limits, NULL is returned instead of
+ * a Const.  All of these structures are freshly palloc'd.
  *
- * Checks if the new partition's bound overlaps any of the existing partitions
- * of parent.  Also performs additional checks as necessary per strategy.
+ * *partexprs_item points to the cell containing the next expression in
+ * the key->partexprs list, or NULL.  It may be advanced upon return.
  */
-void
-check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec)
+static void
+get_range_key_properties(PartitionKey key, int keynum,
+						 PartitionRangeDatum *ldatum,
+						 PartitionRangeDatum *udatum,
+						 ListCell **partexprs_item,
+						 Expr **keyCol,
+						 Const **lower_val, Const **upper_val)
 {
-	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	PartitionBoundInfo boundinfo = partdesc->boundinfo;
-	ParseState *pstate = make_parsestate(NULL);
-	int			with = -1;
-	bool		overlap = false;
-
-	if (spec->is_default)
+	/* Get partition key expression for this column */
+	if (key->partattrs[keynum] != 0)
 	{
-		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
-			return;
-
-		/* Default partition already exists, error out. */
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
-				 parser_errposition(pstate, spec->location)));
+		*keyCol = (Expr *) makeVar(1,
+								   key->partattrs[keynum],
+								   key->parttypid[keynum],
+								   key->parttypmod[keynum],
+								   key->parttypcoll[keynum],
+								   0);
 	}
-
-	switch (key->strategy)
+	else
 	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+		if (*partexprs_item == NULL)
+			elog(ERROR, "wrong number of partition key expressions");
+		*keyCol = copyObject(lfirst(*partexprs_item));
+		*partexprs_item = lnext(*partexprs_item);
+	}
 
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					Datum	  **datums = boundinfo->datums;
-					int			ndatums = boundinfo->ndatums;
-					int			greatest_modulus;
-					int			remainder;
-					int			offset;
-					bool		valid_modulus = true;
-					int			prev_modulus,	/* Previous largest modulus */
-								next_modulus;	/* Next largest modulus */
-
-					/*
-					 * Check rule that every modulus must be a factor of the
-					 * next larger modulus.  For example, if you have a bunch
-					 * of partitions that all have modulus 5, you can add a
-					 * new partition with modulus 10 or a new partition with
-					 * modulus 15, but you cannot add both a partition with
-					 * modulus 10 and a partition with modulus 15, because 10
-					 * is not a factor of 15.
-					 *
-					 * Get the greatest (modulus, remainder) pair contained in
-					 * boundinfo->datums that is less than or equal to the
-					 * (spec->modulus, spec->remainder) pair.
-					 */
-					offset = partition_hash_bsearch(key, boundinfo,
-													spec->modulus,
-													spec->remainder);
-					if (offset < 0)
-					{
-						next_modulus = DatumGetInt32(datums[0][0]);
-						valid_modulus = (next_modulus % spec->modulus) == 0;
-					}
-					else
-					{
-						prev_modulus = DatumGetInt32(datums[offset][0]);
-						valid_modulus = (spec->modulus % prev_modulus) == 0;
-
-						if (valid_modulus && (offset + 1) < ndatums)
-						{
-							next_modulus = DatumGetInt32(datums[offset + 1][0]);
-							valid_modulus = (next_modulus % spec->modulus) == 0;
-						}
-					}
+	/* Get appropriate Const nodes for the bounds */
+	if (ldatum->kind == PARTITION_RANGE_DATUM_VALUE)
+		*lower_val = castNode(Const, copyObject(ldatum->value));
+	else
+		*lower_val = NULL;
 
-					if (!valid_modulus)
-						ereport(ERROR,
-								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
-
-					greatest_modulus = get_greatest_modulus(boundinfo);
-					remainder = spec->remainder;
-
-					/*
-					 * Normally, the lowest remainder that could conflict with
-					 * the new partition is equal to the remainder specified
-					 * for the new partition, but when the new partition has a
-					 * modulus higher than any used so far, we need to adjust.
-					 */
-					if (remainder >= greatest_modulus)
-						remainder = remainder % greatest_modulus;
-
-					/* Check every potentially-conflicting remainder. */
-					do
-					{
-						if (boundinfo->indexes[remainder] != -1)
-						{
-							overlap = true;
-							with = boundinfo->indexes[remainder];
-							break;
-						}
-						remainder += spec->modulus;
-					} while (remainder < greatest_modulus);
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_LIST:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-
-				if (partdesc->nparts > 0)
-				{
-					ListCell   *cell;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo) ||
-							partition_bound_has_default(boundinfo)));
-
-					foreach(cell, spec->listdatums)
-					{
-						Const	   *val = castNode(Const, lfirst(cell));
-
-						if (!val->constisnull)
-						{
-							int			offset;
-							bool		equal;
-
-							offset = partition_list_bsearch(key, boundinfo,
-															val->constvalue,
-															&equal);
-							if (offset >= 0 && equal)
-							{
-								overlap = true;
-								with = boundinfo->indexes[offset];
-								break;
-							}
-						}
-						else if (partition_bound_accepts_nulls(boundinfo))
-						{
-							overlap = true;
-							with = boundinfo->null_index;
-							break;
-						}
-					}
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				PartitionRangeBound *lower,
-						   *upper;
-
-				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
-				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
-
-				/*
-				 * First check if the resulting range would be empty with
-				 * specified lower and upper bounds
-				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
-				{
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-							 errmsg("empty range bound specified for partition \"%s\"",
-									relname),
-							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
-									   get_range_partbound_string(spec->lowerdatums),
-									   get_range_partbound_string(spec->upperdatums)),
-							 parser_errposition(pstate, spec->location)));
-				}
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					int			offset;
-					bool		equal;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_has_default(boundinfo)));
-
-					/*
-					 * Test whether the new lower bound (which is treated
-					 * inclusively as part of the new partition) lies inside
-					 * an existing partition, or in a gap.
-					 *
-					 * If it's inside an existing partition, the bound at
-					 * offset + 1 will be the upper bound of that partition,
-					 * and its index will be >= 0.
-					 *
-					 * If it's in a gap, the bound at offset + 1 will be the
-					 * lower bound of the next partition, and its index will
-					 * be -1. This is also true if there is no next partition,
-					 * since the index array is initialised with an extra -1
-					 * at the end.
-					 */
-					offset = partition_range_bsearch(key, boundinfo, lower,
-													 &equal);
-
-					if (boundinfo->indexes[offset + 1] < 0)
-					{
-						/*
-						 * Check that the new partition will fit in the gap.
-						 * For it to fit, the new upper bound must be less
-						 * than or equal to the lower bound of the next
-						 * partition, if there is one.
-						 */
-						if (offset + 1 < boundinfo->ndatums)
-						{
-							int32		cmpval;
-							Datum 	   *datums;
-							PartitionRangeDatumKind *kind;
-							bool		is_lower;
-
-							datums = boundinfo->datums[offset + 1];
-							kind = boundinfo->kind[offset + 1];
-							is_lower = (boundinfo->indexes[offset + 1] == -1);
-
-							cmpval = partition_rbound_cmp(key, datums, kind,
-														  is_lower, upper);
-							if (cmpval < 0)
-							{
-								/*
-								 * The new partition overlaps with the
-								 * existing partition between offset + 1 and
-								 * offset + 2.
-								 */
-								overlap = true;
-								with = boundinfo->indexes[offset + 2];
-							}
-						}
-					}
-					else
-					{
-						/*
-						 * The new partition overlaps with the existing
-						 * partition between offset and offset + 1.
-						 */
-						overlap = true;
-						with = boundinfo->indexes[offset + 1];
-					}
-				}
-
-				break;
-			}
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	if (overlap)
-	{
-		Assert(with >= 0);
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" would overlap partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[with])),
-				 parser_errposition(pstate, spec->location)));
-	}
-}
-
-/*
- * check_default_allows_bound
- *
- * This function checks if there exists a row in the default partition that
- * would properly belong to the new partition being added.  If it finds one,
- * it throws an error.
- */
-void
-check_default_allows_bound(Relation parent, Relation default_rel,
-						   PartitionBoundSpec *new_spec)
-{
-	List	   *new_part_constraints;
-	List	   *def_part_constraints;
-	List	   *all_parts;
-	ListCell   *lc;
-
-	new_part_constraints = (new_spec->strategy == PARTITION_STRATEGY_LIST)
-		? get_qual_for_list(parent, new_spec)
-		: get_qual_for_range(parent, new_spec, false);
-	def_part_constraints =
-		get_proposed_default_constraint(new_part_constraints);
-
-	/*
-	 * If the existing constraints on the default partition imply that it will
-	 * not contain any row that would belong to the new partition, we can
-	 * avoid scanning the default partition.
-	 */
-	if (PartConstraintImpliedByRelConstraint(default_rel, def_part_constraints))
-	{
-		ereport(INFO,
-				(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
-						RelationGetRelationName(default_rel))));
-		return;
-	}
-
-	/*
-	 * Scan the default partition and its subpartitions, and check for rows
-	 * that do not satisfy the revised partition constraints.
-	 */
-	if (default_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-		all_parts = find_all_inheritors(RelationGetRelid(default_rel),
-										AccessExclusiveLock, NULL);
-	else
-		all_parts = list_make1_oid(RelationGetRelid(default_rel));
-
-	foreach(lc, all_parts)
-	{
-		Oid			part_relid = lfirst_oid(lc);
-		Relation	part_rel;
-		Expr	   *constr;
-		Expr	   *partition_constraint;
-		EState	   *estate;
-		HeapTuple	tuple;
-		ExprState  *partqualstate = NULL;
-		Snapshot	snapshot;
-		TupleDesc	tupdesc;
-		ExprContext *econtext;
-		HeapScanDesc scan;
-		MemoryContext oldCxt;
-		TupleTableSlot *tupslot;
-
-		/* Lock already taken above. */
-		if (part_relid != RelationGetRelid(default_rel))
-		{
-			part_rel = heap_open(part_relid, NoLock);
-
-			/*
-			 * If the partition constraints on default partition child imply
-			 * that it will not contain any row that would belong to the new
-			 * partition, we can avoid scanning the child table.
-			 */
-			if (PartConstraintImpliedByRelConstraint(part_rel,
-													 def_part_constraints))
-			{
-				ereport(INFO,
-						(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
-								RelationGetRelationName(part_rel))));
-
-				heap_close(part_rel, NoLock);
-				continue;
-			}
-		}
-		else
-			part_rel = default_rel;
-
-		/*
-		 * Only RELKIND_RELATION relations (i.e. leaf partitions) need to be
-		 * scanned.
-		 */
-		if (part_rel->rd_rel->relkind != RELKIND_RELATION)
-		{
-			if (part_rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
-				ereport(WARNING,
-						(errcode(ERRCODE_CHECK_VIOLATION),
-						 errmsg("skipped scanning foreign table \"%s\" which is a partition of default partition \"%s\"",
-								RelationGetRelationName(part_rel),
-								RelationGetRelationName(default_rel))));
-
-			if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
-				heap_close(part_rel, NoLock);
-
-			continue;
-		}
-
-		tupdesc = CreateTupleDescCopy(RelationGetDescr(part_rel));
-		constr = linitial(def_part_constraints);
-		partition_constraint = (Expr *)
-			map_partition_varattnos((List *) constr,
-									1, part_rel, parent, NULL);
-		estate = CreateExecutorState();
-
-		/* Build expression execution states for partition check quals */
-		partqualstate = ExecPrepareExpr(partition_constraint, estate);
-
-		econtext = GetPerTupleExprContext(estate);
-		snapshot = RegisterSnapshot(GetLatestSnapshot());
-		scan = heap_beginscan(part_rel, snapshot, 0, NULL);
-		tupslot = MakeSingleTupleTableSlot(tupdesc);
-
-		/*
-		 * Switch to per-tuple memory context and reset it for each tuple
-		 * produced, so we don't leak memory.
-		 */
-		oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
-
-		while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
-		{
-			ExecStoreTuple(tuple, tupslot, InvalidBuffer, false);
-			econtext->ecxt_scantuple = tupslot;
-
-			if (!ExecCheck(partqualstate, econtext))
-				ereport(ERROR,
-						(errcode(ERRCODE_CHECK_VIOLATION),
-						 errmsg("updated partition constraint for default partition \"%s\" would be violated by some row",
-								RelationGetRelationName(default_rel))));
-
-			ResetExprContext(econtext);
-			CHECK_FOR_INTERRUPTS();
-		}
-
-		MemoryContextSwitchTo(oldCxt);
-		heap_endscan(scan);
-		UnregisterSnapshot(snapshot);
-		ExecDropSingleTupleTableSlot(tupslot);
-		FreeExecutorState(estate);
-
-		if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
-			heap_close(part_rel, NoLock);	/* keep the lock until commit */
-	}
-}
-
-/*
- * get_partition_parent
- *
- * Returns inheritance parent of a partition by scanning pg_inherits
- *
- * Note: Because this function assumes that the relation whose OID is passed
- * as an argument will have precisely one parent, it should only be called
- * when it is known that the relation is a partition.
- */
-Oid
-get_partition_parent(Oid relid)
-{
-	Form_pg_inherits form;
-	Relation	catalogRelation;
-	SysScanDesc scan;
-	ScanKeyData key[2];
-	HeapTuple	tuple;
-	Oid			result;
-
-	catalogRelation = heap_open(InheritsRelationId, AccessShareLock);
-
-	ScanKeyInit(&key[0],
-				Anum_pg_inherits_inhrelid,
-				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(relid));
-	ScanKeyInit(&key[1],
-				Anum_pg_inherits_inhseqno,
-				BTEqualStrategyNumber, F_INT4EQ,
-				Int32GetDatum(1));
-
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId, true,
-							  NULL, 2, key);
-
-	tuple = systable_getnext(scan);
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "could not find tuple for parent of relation %u", relid);
-
-	form = (Form_pg_inherits) GETSTRUCT(tuple);
-	result = form->inhparent;
-
-	systable_endscan(scan);
-	heap_close(catalogRelation, AccessShareLock);
-
-	return result;
-}
-
-/*
- * get_qual_from_partbound
- *		Given a parser node for partition bound, return the list of executable
- *		expressions as partition constraint
- */
-List *
-get_qual_from_partbound(Relation rel, Relation parent,
-						PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	List	   *my_qual = NIL;
-
-	Assert(key != NULL);
-
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-			my_qual = get_qual_for_hash(parent, spec);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-			my_qual = get_qual_for_list(parent, spec);
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-			my_qual = get_qual_for_range(parent, spec, false);
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	return my_qual;
-}
-
-/*
- * map_partition_varattnos - maps varattno of any Vars in expr from the
- * attno's of 'from_rel' to the attno's of 'to_rel' partition, each of which
- * may be either a leaf partition or a partitioned table, but both of which
- * must be from the same partitioning hierarchy.
- *
- * Even though all of the same column names must be present in all relations
- * in the hierarchy, and they must also have the same types, the attnos may
- * be different.
- *
- * If found_whole_row is not NULL, *found_whole_row returns whether a
- * whole-row variable was found in the input expression.
- *
- * Note: this will work on any node tree, so really the argument and result
- * should be declared "Node *".  But a substantial majority of the callers
- * are working on Lists, so it's less messy to do the casts internally.
- */
-List *
-map_partition_varattnos(List *expr, int fromrel_varno,
-						Relation to_rel, Relation from_rel,
-						bool *found_whole_row)
-{
-	bool		my_found_whole_row = false;
-
-	if (expr != NIL)
-	{
-		AttrNumber *part_attnos;
-
-		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,
-											fromrel_varno, 0,
-											part_attnos,
-											RelationGetDescr(from_rel)->natts,
-											RelationGetForm(to_rel)->reltype,
-											&my_found_whole_row);
-	}
-
-	if (found_whole_row)
-		*found_whole_row = my_found_whole_row;
-
-	return expr;
-}
-
-/*
- * RelationGetPartitionQual
- *
- * Returns a list of partition quals
- */
-List *
-RelationGetPartitionQual(Relation rel)
-{
-	/* Quick exit */
-	if (!rel->rd_rel->relispartition)
-		return NIL;
-
-	return generate_partition_qual(rel);
-}
-
-/*
- * get_partition_qual_relid
- *
- * Returns an expression tree describing the passed-in relation's partition
- * constraint. If there is no partition constraint returns NULL; this can
- * happen if the default partition is the only partition.
- */
-Expr *
-get_partition_qual_relid(Oid relid)
-{
-	Relation	rel = heap_open(relid, AccessShareLock);
-	Expr	   *result = NULL;
-	List	   *and_args;
-
-	/* Do the work only if this relation is a partition. */
-	if (rel->rd_rel->relispartition)
-	{
-		and_args = generate_partition_qual(rel);
-
-		if (and_args == NIL)
-			result = NULL;
-		else if (list_length(and_args) > 1)
-			result = makeBoolExpr(AND_EXPR, and_args, -1);
-		else
-			result = linitial(and_args);
-	}
-
-	/* Keep the lock. */
-	heap_close(rel, NoLock);
-
-	return result;
-}
-
-/* Module-local functions */
-
-/*
- * get_partition_operator
- *
- * Return oid of the operator of given strategy for a given partition key
- * column.
- */
-static Oid
-get_partition_operator(PartitionKey key, int col, StrategyNumber strategy,
-					   bool *need_relabel)
-{
-	Oid			operoid;
-
-	/*
-	 * First check if there exists an operator of the given strategy, with
-	 * this column's type as both its lefttype and righttype, in the
-	 * partitioning operator family specified for the column.
-	 */
-	operoid = get_opfamily_member(key->partopfamily[col],
-								  key->parttypid[col],
-								  key->parttypid[col],
-								  strategy);
-
-	/*
-	 * If one doesn't exist, we must resort to using an operator in the same
-	 * operator family but with the operator class declared input type.  It is
-	 * OK to do so, because the column's type is known to be binary-coercible
-	 * with the operator class input type (otherwise, the operator class in
-	 * question would not have been accepted as the partitioning operator
-	 * class).  We must however inform the caller to wrap the non-Const
-	 * expression with a RelabelType node to denote the implicit coercion. It
-	 * ensures that the resulting expression structurally matches similarly
-	 * processed expressions within the optimizer.
-	 */
-	if (!OidIsValid(operoid))
-	{
-		operoid = get_opfamily_member(key->partopfamily[col],
-									  key->partopcintype[col],
-									  key->partopcintype[col],
-									  strategy);
-		if (!OidIsValid(operoid))
-			elog(ERROR, "missing operator %d(%u,%u) in opfamily %u",
-				 strategy, key->partopcintype[col], key->partopcintype[col],
-				 key->partopfamily[col]);
-		*need_relabel = true;
-	}
-	else
-		*need_relabel = false;
-
-	return operoid;
-}
-
-/*
- * make_partition_op_expr
- *		Returns an Expr for the given partition key column with arg1 and
- *		arg2 as its leftop and rightop, respectively
- */
-static Expr *
-make_partition_op_expr(PartitionKey key, int keynum,
-					   uint16 strategy, Expr *arg1, Expr *arg2)
-{
-	Oid			operoid;
-	bool		need_relabel = false;
-	Expr	   *result = NULL;
-
-	/* Get the correct btree operator for this partitioning column */
-	operoid = get_partition_operator(key, keynum, strategy, &need_relabel);
-
-	/*
-	 * Chosen operator may be such that the non-Const operand needs to be
-	 * coerced, so apply the same; see the comment in
-	 * get_partition_operator().
-	 */
-	if (!IsA(arg1, Const) &&
-		(need_relabel ||
-		 key->partcollation[keynum] != key->parttypcoll[keynum]))
-		arg1 = (Expr *) makeRelabelType(arg1,
-										key->partopcintype[keynum],
-										-1,
-										key->partcollation[keynum],
-										COERCE_EXPLICIT_CAST);
-
-	/* Generate the actual expression */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_LIST:
-			{
-				List	   *elems = (List *) arg2;
-				int			nelems = list_length(elems);
-
-				Assert(nelems >= 1);
-				Assert(keynum == 0);
-
-				if (nelems > 1 &&
-					!type_is_array(key->parttypid[keynum]))
-				{
-					ArrayExpr  *arrexpr;
-					ScalarArrayOpExpr *saopexpr;
-
-					/* Construct an ArrayExpr for the right-hand inputs */
-					arrexpr = makeNode(ArrayExpr);
-					arrexpr->array_typeid =
-									get_array_type(key->parttypid[keynum]);
-					arrexpr->array_collid = key->parttypcoll[keynum];
-					arrexpr->element_typeid = key->parttypid[keynum];
-					arrexpr->elements = elems;
-					arrexpr->multidims = false;
-					arrexpr->location = -1;
-
-					/* Build leftop = ANY (rightop) */
-					saopexpr = makeNode(ScalarArrayOpExpr);
-					saopexpr->opno = operoid;
-					saopexpr->opfuncid = get_opcode(operoid);
-					saopexpr->useOr = true;
-					saopexpr->inputcollid = key->partcollation[keynum];
-					saopexpr->args = list_make2(arg1, arrexpr);
-					saopexpr->location = -1;
-
-					result = (Expr *) saopexpr;
-				}
-				else
-				{
-					List	   *elemops = NIL;
-					ListCell   *lc;
-
-					foreach (lc, elems)
-					{
-						Expr   *elem = lfirst(lc),
-							   *elemop;
-
-						elemop = make_opclause(operoid,
-											   BOOLOID,
-											   false,
-											   arg1, elem,
-											   InvalidOid,
-											   key->partcollation[keynum]);
-						elemops = lappend(elemops, elemop);
-					}
-
-					result = nelems > 1 ? makeBoolExpr(OR_EXPR, elemops, -1) : linitial(elemops);
-				}
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			result = make_opclause(operoid,
-								   BOOLOID,
-								   false,
-								   arg1, arg2,
-								   InvalidOid,
-								   key->partcollation[keynum]);
-			break;
-
-		default:
-			elog(ERROR, "invalid partitioning strategy");
-			break;
-	}
-
-	return result;
-}
-
-/*
- * get_qual_for_hash
- *
- * Given a list of partition columns, modulus and remainder corresponding to a
- * partition, this function returns CHECK constraint expression Node for that
- * partition.
- *
- * The partition constraint for a hash partition is always a call to the
- * built-in function satisfies_hash_partition().  The first two arguments are
- * the modulus and remainder for the partition; the remaining arguments are the
- * values to be hashed.
- */
-static List *
-get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	FuncExpr   *fexpr;
-	Node	   *relidConst;
-	Node	   *modulusConst;
-	Node	   *remainderConst;
-	List	   *args;
-	ListCell   *partexprs_item;
-	int			i;
-
-	/* Fixed arguments. */
-	relidConst = (Node *) makeConst(OIDOID,
-									-1,
-									InvalidOid,
-									sizeof(Oid),
-									ObjectIdGetDatum(RelationGetRelid(parent)),
-									false,
-									true);
-
-	modulusConst = (Node *) makeConst(INT4OID,
-									  -1,
-									  InvalidOid,
-									  sizeof(int32),
-									  Int32GetDatum(spec->modulus),
-									  false,
-									  true);
-
-	remainderConst = (Node *) makeConst(INT4OID,
-										-1,
-										InvalidOid,
-										sizeof(int32),
-										Int32GetDatum(spec->remainder),
-										false,
-										true);
-
-	args = list_make3(relidConst, modulusConst, remainderConst);
-	partexprs_item = list_head(key->partexprs);
-
-	/* Add an argument for each key column. */
-	for (i = 0; i < key->partnatts; i++)
-	{
-		Node	   *keyCol;
-
-		/* Left operand */
-		if (key->partattrs[i] != 0)
-		{
-			keyCol = (Node *) makeVar(1,
-									  key->partattrs[i],
-									  key->parttypid[i],
-									  key->parttypmod[i],
-									  key->parttypcoll[i],
-									  0);
-		}
-		else
-		{
-			keyCol = (Node *) copyObject(lfirst(partexprs_item));
-			partexprs_item = lnext(partexprs_item);
-		}
-
-		args = lappend(args, keyCol);
-	}
-
-	fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
-						 BOOLOID,
-						 args,
-						 InvalidOid,
-						 InvalidOid,
-						 COERCE_EXPLICIT_CALL);
-
-	return list_make1(fexpr);
-}
-
-/*
- * get_qual_for_list
- *
- * Returns an implicit-AND list of expressions to use as a list partition's
- * constraint, given the partition key and bound structures.
- *
- * The function returns NIL for a default partition when it's the only
- * partition since in that case there is no constraint.
- */
-static List *
-get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	List	   *result;
-	Expr	   *keyCol;
-	Expr	   *opexpr;
-	NullTest   *nulltest;
-	ListCell   *cell;
-	List	   *elems = NIL;
-	bool		list_has_null = false;
-
-	/*
-	 * Only single-column list partitioning is supported, so we are worried
-	 * only about the partition key with index 0.
-	 */
-	Assert(key->partnatts == 1);
-
-	/* Construct Var or expression representing the partition column */
-	if (key->partattrs[0] != 0)
-		keyCol = (Expr *) makeVar(1,
-								  key->partattrs[0],
-								  key->parttypid[0],
-								  key->parttypmod[0],
-								  key->parttypcoll[0],
-								  0);
-	else
-		keyCol = (Expr *) copyObject(linitial(key->partexprs));
-
-	/*
-	 * For default list partition, collect datums for all the partitions. The
-	 * default partition constraint should check that the partition key is
-	 * equal to none of those.
-	 */
-	if (spec->is_default)
-	{
-		int			i;
-		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
-		PartitionBoundInfo boundinfo = pdesc->boundinfo;
-
-		if (boundinfo)
-		{
-			ndatums = boundinfo->ndatums;
-
-			if (partition_bound_accepts_nulls(boundinfo))
-				list_has_null = true;
-		}
-
-		/*
-		 * If default is the only partition, there need not be any partition
-		 * constraint on it.
-		 */
-		if (ndatums == 0 && !list_has_null)
-			return NIL;
-
-		for (i = 0; i < ndatums; i++)
-		{
-			Const	   *val;
-
-			/*
-			 * Construct Const from known-not-null datum.  We must be careful
-			 * to copy the value, because our result has to be able to outlive
-			 * the relcache entry we're copying from.
-			 */
-			val = makeConst(key->parttypid[0],
-							key->parttypmod[0],
-							key->parttypcoll[0],
-							key->parttyplen[0],
-							datumCopy(*boundinfo->datums[i],
-									  key->parttypbyval[0],
-									  key->parttyplen[0]),
-							false,	/* isnull */
-							key->parttypbyval[0]);
-
-			elems = lappend(elems, val);
-		}
-	}
-	else
-	{
-		/*
-		 * Create list of Consts for the allowed values, excluding any nulls.
-		 */
-		foreach(cell, spec->listdatums)
-		{
-			Const	   *val = castNode(Const, lfirst(cell));
-
-			if (val->constisnull)
-				list_has_null = true;
-			else
-				elems = lappend(elems, copyObject(val));
-		}
-	}
-
-	if (elems)
-	{
-		/*
-		 * Generate the operator expression from the non-null partition
-		 * values.
-		 */
-		opexpr = make_partition_op_expr(key, 0, BTEqualStrategyNumber,
-										keyCol, (Expr *) elems);
-	}
-	else
-	{
-		/*
-		 * If there are no partition values, we don't need an operator
-		 * expression.
-		 */
-		opexpr = NULL;
-	}
-
-	if (!list_has_null)
-	{
-		/*
-		 * Gin up a "col IS NOT NULL" test that will be AND'd with the main
-		 * expression.  This might seem redundant, but the partition routing
-		 * machinery needs it.
-		 */
-		nulltest = makeNode(NullTest);
-		nulltest->arg = keyCol;
-		nulltest->nulltesttype = IS_NOT_NULL;
-		nulltest->argisrow = false;
-		nulltest->location = -1;
-
-		result = opexpr ? list_make2(nulltest, opexpr) : list_make1(nulltest);
-	}
-	else
-	{
-		/*
-		 * Gin up a "col IS NULL" test that will be OR'd with the main
-		 * expression.
-		 */
-		nulltest = makeNode(NullTest);
-		nulltest->arg = keyCol;
-		nulltest->nulltesttype = IS_NULL;
-		nulltest->argisrow = false;
-		nulltest->location = -1;
-
-		if (opexpr)
-		{
-			Expr	   *or;
-
-			or = makeBoolExpr(OR_EXPR, list_make2(nulltest, opexpr), -1);
-			result = list_make1(or);
-		}
-		else
-			result = list_make1(nulltest);
-	}
-
-	/*
-	 * Note that, in general, applying NOT to a constraint expression doesn't
-	 * necessarily invert the set of rows it accepts, because NOT (NULL) is
-	 * NULL.  However, the partition constraints we construct here never
-	 * evaluate to NULL, so applying NOT works as intended.
-	 */
-	if (spec->is_default)
-	{
-		result = list_make1(make_ands_explicit(result));
-		result = list_make1(makeBoolExpr(NOT_EXPR, result, -1));
-	}
-
-	return result;
-}
-
-/*
- * get_range_key_properties
- *		Returns range partition key information for a given column
- *
- * This is a subroutine for get_qual_for_range, and its API is pretty
- * specialized to that caller.
- *
- * Constructs an Expr for the key column (returned in *keyCol) and Consts
- * for the lower and upper range limits (returned in *lower_val and
- * *upper_val).  For MINVALUE/MAXVALUE limits, NULL is returned instead of
- * a Const.  All of these structures are freshly palloc'd.
- *
- * *partexprs_item points to the cell containing the next expression in
- * the key->partexprs list, or NULL.  It may be advanced upon return.
- */
-static void
-get_range_key_properties(PartitionKey key, int keynum,
-						 PartitionRangeDatum *ldatum,
-						 PartitionRangeDatum *udatum,
-						 ListCell **partexprs_item,
-						 Expr **keyCol,
-						 Const **lower_val, Const **upper_val)
-{
-	/* Get partition key expression for this column */
-	if (key->partattrs[keynum] != 0)
-	{
-		*keyCol = (Expr *) makeVar(1,
-								   key->partattrs[keynum],
-								   key->parttypid[keynum],
-								   key->parttypmod[keynum],
-								   key->parttypcoll[keynum],
-								   0);
-	}
-	else
-	{
-		if (*partexprs_item == NULL)
-			elog(ERROR, "wrong number of partition key expressions");
-		*keyCol = copyObject(lfirst(*partexprs_item));
-		*partexprs_item = lnext(*partexprs_item);
-	}
-
-	/* Get appropriate Const nodes for the bounds */
-	if (ldatum->kind == PARTITION_RANGE_DATUM_VALUE)
-		*lower_val = castNode(Const, copyObject(ldatum->value));
-	else
-		*lower_val = NULL;
-
-	if (udatum->kind == PARTITION_RANGE_DATUM_VALUE)
-		*upper_val = castNode(Const, copyObject(udatum->value));
-	else
-		*upper_val = NULL;
-}
-
- /*
-  * get_range_nulltest
-  *
-  * A non-default range partition table does not currently allow partition
-  * keys to be null, so emit an IS NOT NULL expression for each key column.
-  */
-static List *
-get_range_nulltest(PartitionKey key)
-{
-	List	   *result = NIL;
-	NullTest   *nulltest;
-	ListCell   *partexprs_item;
-	int			i;
-
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		Expr	   *keyCol;
-
-		if (key->partattrs[i] != 0)
-		{
-			keyCol = (Expr *) makeVar(1,
-									  key->partattrs[i],
-									  key->parttypid[i],
-									  key->parttypmod[i],
-									  key->parttypcoll[i],
-									  0);
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-			keyCol = copyObject(lfirst(partexprs_item));
-			partexprs_item = lnext(partexprs_item);
-		}
-
-		nulltest = makeNode(NullTest);
-		nulltest->arg = keyCol;
-		nulltest->nulltesttype = IS_NOT_NULL;
-		nulltest->argisrow = false;
-		nulltest->location = -1;
-		result = lappend(result, nulltest);
-	}
-
-	return result;
-}
-
-/*
- * get_qual_for_range
- *
- * Returns an implicit-AND list of expressions to use as a range partition's
- * constraint, given the partition key and bound structures.
- *
- * For a multi-column range partition key, say (a, b, c), with (al, bl, cl)
- * as the lower bound tuple and (au, bu, cu) as the upper bound tuple, we
- * generate an expression tree of the following form:
- *
- *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
- *		AND
- *	(a > al OR (a = al AND b > bl) OR (a = al AND b = bl AND c >= cl))
- *		AND
- *	(a < au OR (a = au AND b < bu) OR (a = au AND b = bu AND c < cu))
- *
- * It is often the case that a prefix of lower and upper bound tuples contains
- * the same values, for example, (al = au), in which case, we will emit an
- * expression tree of the following form:
- *
- *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
- *		AND
- *	(a = al)
- *		AND
- *	(b > bl OR (b = bl AND c >= cl))
- *		AND
- *	(b < bu) OR (b = bu AND c < cu))
- *
- * If a bound datum is either MINVALUE or MAXVALUE, these expressions are
- * simplified using the fact that any value is greater than MINVALUE and less
- * than MAXVALUE. So, for example, if cu = MAXVALUE, c < cu is automatically
- * true, and we need not emit any expression for it, and the last line becomes
- *
- *	(b < bu) OR (b = bu), which is simplified to (b <= bu)
- *
- * In most common cases with only one partition column, say a, the following
- * expression tree will be generated: a IS NOT NULL AND a >= al AND a < au
- *
- * For default partition, it returns the negation of the constraints of all
- * the other partitions.
- *
- * External callers should pass for_default as false; we set it to true only
- * when recursing.
- */
-static List *
-get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
-				   bool for_default)
-{
-	List	   *result = NIL;
-	ListCell   *cell1,
-			   *cell2,
-			   *partexprs_item,
-			   *partexprs_item_saved;
-	int			i,
-				j;
-	PartitionRangeDatum *ldatum,
-			   *udatum;
-	PartitionKey key = RelationGetPartitionKey(parent);
-	Expr	   *keyCol;
-	Const	   *lower_val,
-			   *upper_val;
-	List	   *lower_or_arms,
-			   *upper_or_arms;
-	int			num_or_arms,
-				current_or_arm;
-	ListCell   *lower_or_start_datum,
-			   *upper_or_start_datum;
-	bool		need_next_lower_arm,
-				need_next_upper_arm;
-
-	if (spec->is_default)
-	{
-		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
-		Oid		   *inhoids = pdesc->oids;
-		int			nparts = pdesc->nparts,
-					i;
-
-		for (i = 0; i < nparts; i++)
-		{
-			Oid			inhrelid = inhoids[i];
-			HeapTuple	tuple;
-			Datum		datum;
-			bool		isnull;
-			PartitionBoundSpec *bspec;
-
-			tuple = SearchSysCache1(RELOID, inhrelid);
-			if (!HeapTupleIsValid(tuple))
-				elog(ERROR, "cache lookup failed for relation %u", inhrelid);
-
-			datum = SysCacheGetAttr(RELOID, tuple,
-									Anum_pg_class_relpartbound,
-									&isnull);
-
-			Assert(!isnull);
-			bspec = (PartitionBoundSpec *)
-				stringToNode(TextDatumGetCString(datum));
-			if (!IsA(bspec, PartitionBoundSpec))
-				elog(ERROR, "expected PartitionBoundSpec");
-
-			if (!bspec->is_default)
-			{
-				List	   *part_qual;
-
-				part_qual = get_qual_for_range(parent, bspec, true);
-
-				/*
-				 * AND the constraints of the partition and add to
-				 * or_expr_args
-				 */
-				or_expr_args = lappend(or_expr_args, list_length(part_qual) > 1
-									   ? makeBoolExpr(AND_EXPR, part_qual, -1)
-									   : linitial(part_qual));
-			}
-			ReleaseSysCache(tuple);
-		}
-
-		if (or_expr_args != NIL)
-		{
-			Expr	   *other_parts_constr;
-
-			/*
-			 * Combine the constraints obtained for non-default partitions
-			 * using OR.  As requested, each of the OR's args doesn't include
-			 * the NOT NULL test for partition keys (which is to avoid its
-			 * useless repetition).  Add the same now.
-			 */
-			other_parts_constr =
-				makeBoolExpr(AND_EXPR,
-							 lappend(get_range_nulltest(key),
-									 list_length(or_expr_args) > 1
-									 ? makeBoolExpr(OR_EXPR, or_expr_args,
-													-1)
-									 : linitial(or_expr_args)),
-							 -1);
-
-			/*
-			 * Finally, the default partition contains everything *NOT*
-			 * contained in the non-default partitions.
-			 */
-			result = list_make1(makeBoolExpr(NOT_EXPR,
-											 list_make1(other_parts_constr), -1));
-		}
-
-		return result;
-	}
-
-	lower_or_start_datum = list_head(spec->lowerdatums);
-	upper_or_start_datum = list_head(spec->upperdatums);
-	num_or_arms = key->partnatts;
-
-	/*
-	 * If it is the recursive call for default, we skip the get_range_nulltest
-	 * to avoid accumulating the NullTest on the same keys for each partition.
-	 */
-	if (!for_default)
-		result = get_range_nulltest(key);
-
-	/*
-	 * Iterate over the key columns and check if the corresponding lower and
-	 * upper datums are equal using the btree equality operator for the
-	 * column's type.  If equal, we emit single keyCol = common_value
-	 * expression.  Starting from the first column for which the corresponding
-	 * lower and upper bound datums are not equal, we generate OR expressions
-	 * as shown in the function's header comment.
-	 */
-	i = 0;
-	partexprs_item = list_head(key->partexprs);
-	partexprs_item_saved = partexprs_item;	/* placate compiler */
-	forboth(cell1, spec->lowerdatums, cell2, spec->upperdatums)
-	{
-		EState	   *estate;
-		MemoryContext oldcxt;
-		Expr	   *test_expr;
-		ExprState  *test_exprstate;
-		Datum		test_result;
-		bool		isNull;
-
-		ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
-		udatum = castNode(PartitionRangeDatum, lfirst(cell2));
-
-		/*
-		 * Since get_range_key_properties() modifies partexprs_item, and we
-		 * might need to start over from the previous expression in the later
-		 * part of this function, save away the current value.
-		 */
-		partexprs_item_saved = partexprs_item;
-
-		get_range_key_properties(key, i, ldatum, udatum,
-								 &partexprs_item,
-								 &keyCol,
-								 &lower_val, &upper_val);
-
-		/*
-		 * If either value is NULL, the corresponding partition bound is
-		 * either MINVALUE or MAXVALUE, and we treat them as unequal, because
-		 * even if they're the same, there is no common value to equate the
-		 * key column with.
-		 */
-		if (!lower_val || !upper_val)
-			break;
-
-		/* Create the test expression */
-		estate = CreateExecutorState();
-		oldcxt = MemoryContextSwitchTo(estate->es_query_cxt);
-		test_expr = make_partition_op_expr(key, i, BTEqualStrategyNumber,
-										   (Expr *) lower_val,
-										   (Expr *) upper_val);
-		fix_opfuncids((Node *) test_expr);
-		test_exprstate = ExecInitExpr(test_expr, NULL);
-		test_result = ExecEvalExprSwitchContext(test_exprstate,
-												GetPerTupleExprContext(estate),
-												&isNull);
-		MemoryContextSwitchTo(oldcxt);
-		FreeExecutorState(estate);
-
-		/* If not equal, go generate the OR expressions */
-		if (!DatumGetBool(test_result))
-			break;
-
-		/*
-		 * The bounds for the last key column can't be equal, because such a
-		 * range partition would never be allowed to be defined (it would have
-		 * an empty range otherwise).
-		 */
-		if (i == key->partnatts - 1)
-			elog(ERROR, "invalid range bound specification");
-
-		/* Equal, so generate keyCol = lower_val expression */
-		result = lappend(result,
-						 make_partition_op_expr(key, i, BTEqualStrategyNumber,
-												keyCol, (Expr *) lower_val));
-
-		i++;
-	}
-
-	/* First pair of lower_val and upper_val that are not equal. */
-	lower_or_start_datum = cell1;
-	upper_or_start_datum = cell2;
-
-	/* OR will have as many arms as there are key columns left. */
-	num_or_arms = key->partnatts - i;
-	current_or_arm = 0;
-	lower_or_arms = upper_or_arms = NIL;
-	need_next_lower_arm = need_next_upper_arm = true;
-	while (current_or_arm < num_or_arms)
-	{
-		List	   *lower_or_arm_args = NIL,
-				   *upper_or_arm_args = NIL;
-
-		/* Restart scan of columns from the i'th one */
-		j = i;
-		partexprs_item = partexprs_item_saved;
-
-		for_both_cell(cell1, lower_or_start_datum, cell2, upper_or_start_datum)
-		{
-			PartitionRangeDatum *ldatum_next = NULL,
-					   *udatum_next = NULL;
-
-			ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
-			if (lnext(cell1))
-				ldatum_next = castNode(PartitionRangeDatum,
-									   lfirst(lnext(cell1)));
-			udatum = castNode(PartitionRangeDatum, lfirst(cell2));
-			if (lnext(cell2))
-				udatum_next = castNode(PartitionRangeDatum,
-									   lfirst(lnext(cell2)));
-			get_range_key_properties(key, j, ldatum, udatum,
-									 &partexprs_item,
-									 &keyCol,
-									 &lower_val, &upper_val);
-
-			if (need_next_lower_arm && lower_val)
-			{
-				uint16		strategy;
-
-				/*
-				 * For the non-last columns of this arm, use the EQ operator.
-				 * For the last column of this arm, use GT, unless this is the
-				 * last column of the whole bound check, or the next bound
-				 * datum is MINVALUE, in which case use GE.
-				 */
-				if (j - i < current_or_arm)
-					strategy = BTEqualStrategyNumber;
-				else if (j == key->partnatts - 1 ||
-						 (ldatum_next &&
-						  ldatum_next->kind == PARTITION_RANGE_DATUM_MINVALUE))
-					strategy = BTGreaterEqualStrategyNumber;
-				else
-					strategy = BTGreaterStrategyNumber;
-
-				lower_or_arm_args = lappend(lower_or_arm_args,
-											make_partition_op_expr(key, j,
-																   strategy,
-																   keyCol,
-																   (Expr *) lower_val));
-			}
-
-			if (need_next_upper_arm && upper_val)
-			{
-				uint16		strategy;
-
-				/*
-				 * For the non-last columns of this arm, use the EQ operator.
-				 * For the last column of this arm, use LT, unless the next
-				 * bound datum is MAXVALUE, in which case use LE.
-				 */
-				if (j - i < current_or_arm)
-					strategy = BTEqualStrategyNumber;
-				else if (udatum_next &&
-						 udatum_next->kind == PARTITION_RANGE_DATUM_MAXVALUE)
-					strategy = BTLessEqualStrategyNumber;
-				else
-					strategy = BTLessStrategyNumber;
-
-				upper_or_arm_args = lappend(upper_or_arm_args,
-											make_partition_op_expr(key, j,
-																   strategy,
-																   keyCol,
-																   (Expr *) upper_val));
-			}
-
-			/*
-			 * Did we generate enough of OR's arguments?  First arm considers
-			 * the first of the remaining columns, second arm considers first
-			 * two of the remaining columns, and so on.
-			 */
-			++j;
-			if (j - i > current_or_arm)
-			{
-				/*
-				 * We must not emit any more arms if the new column that will
-				 * be considered is unbounded, or this one was.
-				 */
-				if (!lower_val || !ldatum_next ||
-					ldatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
-					need_next_lower_arm = false;
-				if (!upper_val || !udatum_next ||
-					udatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
-					need_next_upper_arm = false;
-				break;
-			}
-		}
-
-		if (lower_or_arm_args != NIL)
-			lower_or_arms = lappend(lower_or_arms,
-									list_length(lower_or_arm_args) > 1
-									? makeBoolExpr(AND_EXPR, lower_or_arm_args, -1)
-									: linitial(lower_or_arm_args));
-
-		if (upper_or_arm_args != NIL)
-			upper_or_arms = lappend(upper_or_arms,
-									list_length(upper_or_arm_args) > 1
-									? makeBoolExpr(AND_EXPR, upper_or_arm_args, -1)
-									: linitial(upper_or_arm_args));
-
-		/* If no work to do in the next iteration, break away. */
-		if (!need_next_lower_arm && !need_next_upper_arm)
-			break;
-
-		++current_or_arm;
-	}
-
-	/*
-	 * Generate the OR expressions for each of lower and upper bounds (if
-	 * required), and append to the list of implicitly ANDed list of
-	 * expressions.
-	 */
-	if (lower_or_arms != NIL)
-		result = lappend(result,
-						 list_length(lower_or_arms) > 1
-						 ? makeBoolExpr(OR_EXPR, lower_or_arms, -1)
-						 : linitial(lower_or_arms));
-	if (upper_or_arms != NIL)
-		result = lappend(result,
-						 list_length(upper_or_arms) > 1
-						 ? makeBoolExpr(OR_EXPR, upper_or_arms, -1)
-						 : linitial(upper_or_arms));
-
-	/*
-	 * As noted above, for non-default, we return list with constant TRUE. If
-	 * the result is NIL during the recursive call for default, it implies
-	 * this is the only other partition which can hold every value of the key
-	 * except NULL. Hence we return the NullTest result skipped earlier.
-	 */
-	if (result == NIL)
-		result = for_default
-			? get_range_nulltest(key)
-			: list_make1(makeBoolConst(true, false));
-
-	return result;
-}
-
-/*
- * generate_partition_qual
- *
- * Generate partition predicate from rel's partition bound expression. The
- * function returns a NIL list if there is no predicate.
- *
- * Result expression tree is stored CacheMemoryContext to ensure it survives
- * as long as the relcache entry. But we should be running in a less long-lived
- * working context. To avoid leaking cache memory if this routine fails partway
- * through, we build in working memory and then copy the completed structure
- * into cache memory.
- */
-static List *
-generate_partition_qual(Relation rel)
-{
-	HeapTuple	tuple;
-	MemoryContext oldcxt;
-	Datum		boundDatum;
-	bool		isnull;
-	PartitionBoundSpec *bound;
-	List	   *my_qual = NIL,
-			   *result = NIL;
-	Relation	parent;
-	bool		found_whole_row;
-
-	/* Guard against stack overflow due to overly deep partition tree */
-	check_stack_depth();
-
-	/* Quick copy */
-	if (rel->rd_partcheck != NIL)
-		return copyObject(rel->rd_partcheck);
-
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
-					   AccessShareLock);
-
-	/* Get pg_class.relpartbound */
-	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(rel));
-
-	boundDatum = SysCacheGetAttr(RELOID, tuple,
-								 Anum_pg_class_relpartbound,
-								 &isnull);
-	if (isnull)					/* should not happen */
-		elog(ERROR, "relation \"%s\" has relpartbound = null",
-			 RelationGetRelationName(rel));
-	bound = castNode(PartitionBoundSpec,
-					 stringToNode(TextDatumGetCString(boundDatum)));
-	ReleaseSysCache(tuple);
-
-	my_qual = get_qual_from_partbound(rel, parent, bound);
-
-	/* Add the parent's quals to the list (if any) */
-	if (parent->rd_rel->relispartition)
-		result = list_concat(generate_partition_qual(parent), my_qual);
+	if (udatum->kind == PARTITION_RANGE_DATUM_VALUE)
+		*upper_val = castNode(Const, copyObject(udatum->value));
 	else
-		result = my_qual;
-
-	/*
-	 * Change Vars to have partition's attnos instead of the parent's. We do
-	 * this after we concatenate the parent's quals, because we want every Var
-	 * in it to bear this relation's attnos. It's safe to assume varno = 1
-	 * here.
-	 */
-	result = map_partition_varattnos(result, 1, rel, parent,
-									 &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");
-
-	/* Save a copy in the relcache */
-	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
-	rel->rd_partcheck = copyObject(result);
-	MemoryContextSwitchTo(oldcxt);
-
-	/* Keep the parent locked until commit */
-	heap_close(parent, NoLock);
-
-	return result;
-}
-
-/*
- * get_partition_for_tuple
- *		Finds partition of relation which accepts the partition key specified
- *		in values and isnull
- *
- * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
- * found or -1 if none found.
- */
-int
-get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
-{
-	int			bound_offset;
-	int			part_index = -1;
-	PartitionKey key = RelationGetPartitionKey(relation);
-	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
-
-	/* Route as appropriate based on partitioning strategy. */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				PartitionBoundInfo boundinfo = partdesc->boundinfo;
-				int			greatest_modulus = get_greatest_modulus(boundinfo);
-				uint64		rowHash = compute_hash_value(key, values, isnull);
-
-				part_index = boundinfo->indexes[rowHash % greatest_modulus];
-			}
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			if (isnull[0])
-			{
-				if (partition_bound_accepts_nulls(partdesc->boundinfo))
-					part_index = partdesc->boundinfo->null_index;
-			}
-			else
-			{
-				bool		equal = false;
-
-				bound_offset = partition_list_bsearch(key,
-													  partdesc->boundinfo,
-													  values[0], &equal);
-				if (bound_offset >= 0 && equal)
-					part_index = partdesc->boundinfo->indexes[bound_offset];
-			}
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				bool		equal = false,
-							range_partkey_has_null = false;
-				int			i;
-
-				/*
-				 * No range includes NULL, so this will be accepted by the
-				 * default partition if there is one, and otherwise rejected.
-				 */
-				for (i = 0; i < key->partnatts; i++)
-				{
-					if (isnull[i])
-					{
-						range_partkey_has_null = true;
-						break;
-					}
-				}
-
-				if (!range_partkey_has_null)
-				{
-					bound_offset = partition_range_datum_bsearch(key,
-														partdesc->boundinfo,
-														key->partnatts,
-														values,
-														&equal);
-					/*
-					 * The bound at bound_offset is less than or equal to the
-					 * tuple value, so the bound at offset+1 is the upper
-					 * bound of the partition we're looking for, if there
-					 * actually exists one.
-					 */
-					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
-				}
-			}
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/*
-	 * part_index < 0 means we failed to find a partition of this parent. Use
-	 * the default partition, if there is one.
-	 */
-	if (part_index < 0)
-		part_index = partdesc->boundinfo->default_index;
-
-	return part_index;
+		*upper_val = NULL;
 }
 
-/*
- * Checks if any of the 'attnums' is a partition key attribute for rel
- *
- * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
- * partition key expression.  It's possible for a column to be both used
- * directly and as part of an expression; if that happens, *used_in_expr may
- * end up as either true or false.  That's OK for current uses of this
- * function, because *used_in_expr is only used to tailor the error message
- * text.
- */
-bool
-has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr)
+ /*
+  * get_range_nulltest
+  *
+  * A non-default range partition table does not currently allow partition
+  * keys to be null, so emit an IS NOT NULL expression for each key column.
+  */
+static List *
+get_range_nulltest(PartitionKey key)
 {
-	PartitionKey key;
-	int			partnatts;
-	List	   *partexprs;
+	List	   *result = NIL;
+	NullTest   *nulltest;
 	ListCell   *partexprs_item;
 	int			i;
 
-	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return false;
-
-	key = RelationGetPartitionKey(rel);
-	partnatts = get_partition_natts(key);
-	partexprs = get_partition_exprs(key);
-
-	partexprs_item = list_head(partexprs);
-	for (i = 0; i < partnatts; i++)
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
 	{
-		AttrNumber	partattno = get_partition_col_attnum(key, i);
+		Expr	   *keyCol;
 
-		if (partattno != 0)
+		if (key->partattrs[i] != 0)
 		{
-			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
-							  attnums))
-			{
-				if (used_in_expr)
-					*used_in_expr = false;
-				return true;
-			}
+			keyCol = (Expr *) makeVar(1,
+									  key->partattrs[i],
+									  key->parttypid[i],
+									  key->parttypmod[i],
+									  key->parttypcoll[i],
+									  0);
 		}
 		else
 		{
-			/* Arbitrary expression */
-			Node	   *expr = (Node *) lfirst(partexprs_item);
-			Bitmapset  *expr_attrs = NULL;
-
-			/* Find all attributes referenced */
-			pull_varattnos(expr, 1, &expr_attrs);
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+			keyCol = copyObject(lfirst(partexprs_item));
 			partexprs_item = lnext(partexprs_item);
-
-			if (bms_overlap(attnums, expr_attrs))
-			{
-				if (used_in_expr)
-					*used_in_expr = true;
-				return true;
-			}
 		}
+
+		nulltest = makeNode(NullTest);
+		nulltest->arg = keyCol;
+		nulltest->nulltesttype = IS_NOT_NULL;
+		nulltest->argisrow = false;
+		nulltest->location = -1;
+		result = lappend(result, nulltest);
 	}
 
-	return false;
+	return result;
 }
 
 /*
- * qsort_partition_hbound_cmp
+ * get_qual_for_range
  *
- * We sort hash bounds by modulus, then by remainder.
- */
-static int32
-qsort_partition_hbound_cmp(const void *a, const void *b)
-{
-	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
-	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
-
-	return partition_hbound_cmp(h1->modulus, h1->remainder,
-								h2->modulus, h2->remainder);
-}
-
-/*
- * partition_hbound_cmp
+ * Returns an implicit-AND list of expressions to use as a range partition's
+ * constraint, given the partition key and bound structures.
  *
- * Compares modulus first, then remainder if modulus are equal.
- */
-static int32
-partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
-{
-	if (modulus1 < modulus2)
-		return -1;
-	if (modulus1 > modulus2)
-		return 1;
-	if (modulus1 == modulus2 && remainder1 != remainder2)
-		return (remainder1 > remainder2) ? 1 : -1;
-	return 0;
-}
-
-/*
- * qsort_partition_list_value_cmp
+ * For a multi-column range partition key, say (a, b, c), with (al, bl, cl)
+ * as the lower bound tuple and (au, bu, cu) as the upper bound tuple, we
+ * generate an expression tree of the following form:
+ *
+ *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
+ *		AND
+ *	(a > al OR (a = al AND b > bl) OR (a = al AND b = bl AND c >= cl))
+ *		AND
+ *	(a < au OR (a = au AND b < bu) OR (a = au AND b = bu AND c < cu))
+ *
+ * It is often the case that a prefix of lower and upper bound tuples contains
+ * the same values, for example, (al = au), in which case, we will emit an
+ * expression tree of the following form:
+ *
+ *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
+ *		AND
+ *	(a = al)
+ *		AND
+ *	(b > bl OR (b = bl AND c >= cl))
+ *		AND
+ *	(b < bu) OR (b = bu AND c < cu))
+ *
+ * If a bound datum is either MINVALUE or MAXVALUE, these expressions are
+ * simplified using the fact that any value is greater than MINVALUE and less
+ * than MAXVALUE. So, for example, if cu = MAXVALUE, c < cu is automatically
+ * true, and we need not emit any expression for it, and the last line becomes
+ *
+ *	(b < bu) OR (b = bu), which is simplified to (b <= bu)
+ *
+ * In most common cases with only one partition column, say a, the following
+ * expression tree will be generated: a IS NOT NULL AND a >= al AND a < au
  *
- * Compare two list partition bound datums
+ * For default partition, it returns the negation of the constraints of all
+ * the other partitions.
+ *
+ * External callers should pass for_default as false; we set it to true only
+ * when recursing.
  */
-static int32
-qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+static List *
+get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
+				   bool for_default)
 {
-	Datum		val1 = (*(const PartitionListValue **) a)->value,
-				val2 = (*(const PartitionListValue **) b)->value;
-	PartitionKey key = (PartitionKey) arg;
+	List	   *result = NIL;
+	ListCell   *cell1,
+			   *cell2,
+			   *partexprs_item,
+			   *partexprs_item_saved;
+	int			i,
+				j;
+	PartitionRangeDatum *ldatum,
+			   *udatum;
+	PartitionKey key = RelationGetPartitionKey(parent);
+	Expr	   *keyCol;
+	Const	   *lower_val,
+			   *upper_val;
+	List	   *lower_or_arms,
+			   *upper_or_arms;
+	int			num_or_arms,
+				current_or_arm;
+	ListCell   *lower_or_start_datum,
+			   *upper_or_start_datum;
+	bool		need_next_lower_arm,
+				need_next_upper_arm;
 
-	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-										   key->partcollation[0],
-										   val1, val2));
-}
+	if (spec->is_default)
+	{
+		List	   *or_expr_args = NIL;
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		Oid		   *inhoids = pdesc->oids;
+		int			nparts = pdesc->nparts,
+					i;
 
-/*
- * make_one_range_bound
- *
- * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
- * and a flag telling whether the bound is lower or not.  Made into a function
- * because there are multiple sites that want to use this facility.
- */
-static PartitionRangeBound *
-make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
-{
-	PartitionRangeBound *bound;
-	ListCell   *lc;
-	int			i;
+		for (i = 0; i < nparts; i++)
+		{
+			Oid			inhrelid = inhoids[i];
+			HeapTuple	tuple;
+			Datum		datum;
+			bool		isnull;
+			PartitionBoundSpec *bspec;
 
-	Assert(datums != NIL);
+			tuple = SearchSysCache1(RELOID, inhrelid);
+			if (!HeapTupleIsValid(tuple))
+				elog(ERROR, "cache lookup failed for relation %u", inhrelid);
 
-	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
-	bound->index = index;
-	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
-	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
-													  sizeof(PartitionRangeDatumKind));
-	bound->lower = lower;
+			datum = SysCacheGetAttr(RELOID, tuple,
+									Anum_pg_class_relpartbound,
+									&isnull);
 
-	i = 0;
-	foreach(lc, datums)
-	{
-		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+			Assert(!isnull);
+			bspec = (PartitionBoundSpec *)
+				stringToNode(TextDatumGetCString(datum));
+			if (!IsA(bspec, PartitionBoundSpec))
+				elog(ERROR, "expected PartitionBoundSpec");
+
+			if (!bspec->is_default)
+			{
+				List	   *part_qual;
+
+				part_qual = get_qual_for_range(parent, bspec, true);
 
-		/* What's contained in this range datum? */
-		bound->kind[i] = datum->kind;
+				/*
+				 * AND the constraints of the partition and add to
+				 * or_expr_args
+				 */
+				or_expr_args = lappend(or_expr_args, list_length(part_qual) > 1
+									   ? makeBoolExpr(AND_EXPR, part_qual, -1)
+									   : linitial(part_qual));
+			}
+			ReleaseSysCache(tuple);
+		}
 
-		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		if (or_expr_args != NIL)
 		{
-			Const	   *val = castNode(Const, datum->value);
+			Expr	   *other_parts_constr;
 
-			if (val->constisnull)
-				elog(ERROR, "invalid range bound datum");
-			bound->datums[i] = val->constvalue;
+			/*
+			 * Combine the constraints obtained for non-default partitions
+			 * using OR.  As requested, each of the OR's args doesn't include
+			 * the NOT NULL test for partition keys (which is to avoid its
+			 * useless repetition).  Add the same now.
+			 */
+			other_parts_constr =
+				makeBoolExpr(AND_EXPR,
+							 lappend(get_range_nulltest(key),
+									 list_length(or_expr_args) > 1
+									 ? makeBoolExpr(OR_EXPR, or_expr_args,
+													-1)
+									 : linitial(or_expr_args)),
+							 -1);
+
+			/*
+			 * Finally, the default partition contains everything *NOT*
+			 * contained in the non-default partitions.
+			 */
+			result = list_make1(makeBoolExpr(NOT_EXPR,
+											 list_make1(other_parts_constr), -1));
 		}
 
-		i++;
+		return result;
 	}
 
-	return bound;
-}
+	lower_or_start_datum = list_head(spec->lowerdatums);
+	upper_or_start_datum = list_head(spec->upperdatums);
+	num_or_arms = key->partnatts;
 
-/* Used when sorting range bounds across all range partitions */
-static int32
-qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
-{
-	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
-	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
-	PartitionKey key = (PartitionKey) arg;
+	/*
+	 * If it is the recursive call for default, we skip the get_range_nulltest
+	 * to avoid accumulating the NullTest on the same keys for each partition.
+	 */
+	if (!for_default)
+		result = get_range_nulltest(key);
 
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
-}
+	/*
+	 * Iterate over the key columns and check if the corresponding lower and
+	 * upper datums are equal using the btree equality operator for the
+	 * column's type.  If equal, we emit single keyCol = common_value
+	 * expression.  Starting from the first column for which the corresponding
+	 * lower and upper bound datums are not equal, we generate OR expressions
+	 * as shown in the function's header comment.
+	 */
+	i = 0;
+	partexprs_item = list_head(key->partexprs);
+	partexprs_item_saved = partexprs_item;	/* placate compiler */
+	forboth(cell1, spec->lowerdatums, cell2, spec->upperdatums)
+	{
+		EState	   *estate;
+		MemoryContext oldcxt;
+		Expr	   *test_expr;
+		ExprState  *test_exprstate;
+		Datum		test_result;
+		bool		isNull;
 
-/*
- * partition_rbound_cmp
- *
- * Return for two range bounds whether the 1st one (specified in datums1,
- * kind1, and lower1) is <, =, or > the bound specified in *b2.
- *
- * Note that if the values of the two range bounds compare equal, then we take
- * into account whether they are upper or lower bounds, and an upper bound is
- * considered to be smaller than a lower bound. This is important to the way
- * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
- * structure, which only stores the upper bound of a common boundary between
- * two contiguous partitions.
- */
-static int32
-partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2)
-{
-	int32		cmpval = 0;		/* placate compiler */
-	int			i;
-	Datum	   *datums2 = b2->datums;
-	PartitionRangeDatumKind *kind2 = b2->kind;
-	bool		lower2 = b2->lower;
+		ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
+		udatum = castNode(PartitionRangeDatum, lfirst(cell2));
+
+		/*
+		 * Since get_range_key_properties() modifies partexprs_item, and we
+		 * might need to start over from the previous expression in the later
+		 * part of this function, save away the current value.
+		 */
+		partexprs_item_saved = partexprs_item;
+
+		get_range_key_properties(key, i, ldatum, udatum,
+								 &partexprs_item,
+								 &keyCol,
+								 &lower_val, &upper_val);
+
+		/*
+		 * If either value is NULL, the corresponding partition bound is
+		 * either MINVALUE or MAXVALUE, and we treat them as unequal, because
+		 * even if they're the same, there is no common value to equate the
+		 * key column with.
+		 */
+		if (!lower_val || !upper_val)
+			break;
+
+		/* Create the test expression */
+		estate = CreateExecutorState();
+		oldcxt = MemoryContextSwitchTo(estate->es_query_cxt);
+		test_expr = make_partition_op_expr(key, i, BTEqualStrategyNumber,
+										   (Expr *) lower_val,
+										   (Expr *) upper_val);
+		fix_opfuncids((Node *) test_expr);
+		test_exprstate = ExecInitExpr(test_expr, NULL);
+		test_result = ExecEvalExprSwitchContext(test_exprstate,
+												GetPerTupleExprContext(estate),
+												&isNull);
+		MemoryContextSwitchTo(oldcxt);
+		FreeExecutorState(estate);
+
+		/* If not equal, go generate the OR expressions */
+		if (!DatumGetBool(test_result))
+			break;
 
-	for (i = 0; i < key->partnatts; i++)
-	{
 		/*
-		 * First, handle cases where the column is unbounded, which should not
-		 * invoke the comparison procedure, and should not consider any later
-		 * columns. Note that the PartitionRangeDatumKind enum elements
-		 * compare the same way as the values they represent.
+		 * The bounds for the last key column can't be equal, because such a
+		 * range partition would never be allowed to be defined (it would have
+		 * an empty range otherwise).
 		 */
-		if (kind1[i] < kind2[i])
-			return -1;
-		else if (kind1[i] > kind2[i])
-			return 1;
-		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+		if (i == key->partnatts - 1)
+			elog(ERROR, "invalid range bound specification");
 
-			/*
-			 * The column bounds are both MINVALUE or both MAXVALUE. No later
-			 * columns should be considered, but we still need to compare
-			 * whether they are upper or lower bounds.
-			 */
-			break;
+		/* Equal, so generate keyCol = lower_val expression */
+		result = lappend(result,
+						 make_partition_op_expr(key, i, BTEqualStrategyNumber,
+												keyCol, (Expr *) lower_val));
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 datums1[i],
-												 datums2[i]));
-		if (cmpval != 0)
-			break;
+		i++;
 	}
 
-	/*
-	 * If the comparison is anything other than equal, we're done. If they
-	 * compare equal though, we still have to consider whether the boundaries
-	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
-	 * two.
-	 */
-	if (cmpval == 0 && lower1 != lower2)
-		cmpval = lower1 ? 1 : -1;
-
-	return cmpval;
-}
-
-/*
- * partition_rbound_datum_cmp
- *
- * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
- * is <, =, or > partition key of tuple (tuple_datums)
- */
-static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums)
-{
-	int			i;
-	int32		cmpval = -1;
+	/* First pair of lower_val and upper_val that are not equal. */
+	lower_or_start_datum = cell1;
+	upper_or_start_datum = cell2;
 
-	for (i = 0; i < n_tuple_datums; i++)
+	/* OR will have as many arms as there are key columns left. */
+	num_or_arms = key->partnatts - i;
+	current_or_arm = 0;
+	lower_or_arms = upper_or_arms = NIL;
+	need_next_lower_arm = need_next_upper_arm = true;
+	while (current_or_arm < num_or_arms)
 	{
-		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
-			return -1;
-		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
-			return 1;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 rb_datums[i],
-												 tuple_datums[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	return cmpval;
-}
+		List	   *lower_or_arm_args = NIL,
+				   *upper_or_arm_args = NIL;
 
-/*
- * partition_list_bsearch
- *		Returns the index of the greatest bound datum that is less than equal
- * 		to the given value or -1 if all of the bound datums are greater
- *
- * *is_equal is set to true if the bound datum at the returned index is equal
- * to the input value.
- */
-static int
-partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
+		/* Restart scan of columns from the i'th one */
+		j = i;
+		partexprs_item = partexprs_item_saved;
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-												 key->partcollation[0],
-												 boundinfo->datums[mid][0],
-												 value));
-		if (cmpval <= 0)
+		for_both_cell(cell1, lower_or_start_datum, cell2, upper_or_start_datum)
 		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
+			PartitionRangeDatum *ldatum_next = NULL,
+					   *udatum_next = NULL;
 
-	return lo;
-}
+			ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
+			if (lnext(cell1))
+				ldatum_next = castNode(PartitionRangeDatum,
+									   lfirst(lnext(cell1)));
+			udatum = castNode(PartitionRangeDatum, lfirst(cell2));
+			if (lnext(cell2))
+				udatum_next = castNode(PartitionRangeDatum,
+									   lfirst(lnext(cell2)));
+			get_range_key_properties(key, j, ldatum, udatum,
+									 &partexprs_item,
+									 &keyCol,
+									 &lower_val, &upper_val);
 
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given range bound or -1 if all of the range bounds are
- *		greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input range bound
- */
-static int
-partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
+			if (need_next_lower_arm && lower_val)
+			{
+				uint16		strategy;
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_cmp(key,
-									  boundinfo->datums[mid],
-									  boundinfo->kind[mid],
-									  (boundinfo->indexes[mid] == -1),
-									  probe);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
+				/*
+				 * For the non-last columns of this arm, use the EQ operator.
+				 * For the last column of this arm, use GT, unless this is the
+				 * last column of the whole bound check, or the next bound
+				 * datum is MINVALUE, in which case use GE.
+				 */
+				if (j - i < current_or_arm)
+					strategy = BTEqualStrategyNumber;
+				else if (j == key->partnatts - 1 ||
+						 (ldatum_next &&
+						  ldatum_next->kind == PARTITION_RANGE_DATUM_MINVALUE))
+					strategy = BTGreaterEqualStrategyNumber;
+				else
+					strategy = BTGreaterStrategyNumber;
 
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
+				lower_or_arm_args = lappend(lower_or_arm_args,
+											make_partition_op_expr(key, j,
+																   strategy,
+																   keyCol,
+																   (Expr *) lower_val));
+			}
 
-	return lo;
-}
+			if (need_next_upper_arm && upper_val)
+			{
+				uint16		strategy;
 
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given tuple or -1 if all of the range bounds are greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input tuple.
- */
-static int
-partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
+				/*
+				 * For the non-last columns of this arm, use the EQ operator.
+				 * For the last column of this arm, use LT, unless the next
+				 * bound datum is MAXVALUE, in which case use LE.
+				 */
+				if (j - i < current_or_arm)
+					strategy = BTEqualStrategyNumber;
+				else if (udatum_next &&
+						 udatum_next->kind == PARTITION_RANGE_DATUM_MAXVALUE)
+					strategy = BTLessEqualStrategyNumber;
+				else
+					strategy = BTLessStrategyNumber;
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_datum_cmp(key,
-											boundinfo->datums[mid],
-											boundinfo->kind[mid],
-											values,
-											nvalues);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
+				upper_or_arm_args = lappend(upper_or_arm_args,
+											make_partition_op_expr(key, j,
+																   strategy,
+																   keyCol,
+																   (Expr *) upper_val));
+			}
 
-			if (*is_equal)
+			/*
+			 * Did we generate enough of OR's arguments?  First arm considers
+			 * the first of the remaining columns, second arm considers first
+			 * two of the remaining columns, and so on.
+			 */
+			++j;
+			if (j - i > current_or_arm)
+			{
+				/*
+				 * We must not emit any more arms if the new column that will
+				 * be considered is unbounded, or this one was.
+				 */
+				if (!lower_val || !ldatum_next ||
+					ldatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
+					need_next_lower_arm = false;
+				if (!upper_val || !udatum_next ||
+					udatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
+					need_next_upper_arm = false;
 				break;
+			}
 		}
-		else
-			hi = mid - 1;
-	}
 
-	return lo;
-}
+		if (lower_or_arm_args != NIL)
+			lower_or_arms = lappend(lower_or_arms,
+									list_length(lower_or_arm_args) > 1
+									? makeBoolExpr(AND_EXPR, lower_or_arm_args, -1)
+									: linitial(lower_or_arm_args));
 
-/*
- * partition_hash_bsearch
- *		Returns the index of the greatest (modulus, remainder) pair that is
- *		less than or equal to the given (modulus, remainder) pair or -1 if
- *		all of them are greater
- */
-static int
-partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder)
-{
-	int			lo,
-				hi,
-				mid;
+		if (upper_or_arm_args != NIL)
+			upper_or_arms = lappend(upper_or_arms,
+									list_length(upper_or_arm_args) > 1
+									? makeBoolExpr(AND_EXPR, upper_or_arm_args, -1)
+									: linitial(upper_or_arm_args));
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval,
-					bound_modulus,
-					bound_remainder;
-
-		mid = (lo + hi + 1) / 2;
-		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
-		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
-		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
-									  modulus, remainder);
-		if (cmpval <= 0)
-		{
-			lo = mid;
+		/* If no work to do in the next iteration, break away. */
+		if (!need_next_lower_arm && !need_next_upper_arm)
+			break;
 
-			if (cmpval == 0)
-				break;
-		}
-		else
-			hi = mid - 1;
+		++current_or_arm;
 	}
 
-	return lo;
-}
+	/*
+	 * Generate the OR expressions for each of lower and upper bounds (if
+	 * required), and append to the list of implicitly ANDed list of
+	 * expressions.
+	 */
+	if (lower_or_arms != NIL)
+		result = lappend(result,
+						 list_length(lower_or_arms) > 1
+						 ? makeBoolExpr(OR_EXPR, lower_or_arms, -1)
+						 : linitial(lower_or_arms));
+	if (upper_or_arms != NIL)
+		result = lappend(result,
+						 list_length(upper_or_arms) > 1
+						 ? makeBoolExpr(OR_EXPR, upper_or_arms, -1)
+						 : linitial(upper_or_arms));
 
-/*
- * get_default_oid_from_partdesc
- *
- * Given a partition descriptor, return the OID of the default partition, if
- * one exists; else, return InvalidOid.
- */
-Oid
-get_default_oid_from_partdesc(PartitionDesc partdesc)
-{
-	if (partdesc && partdesc->boundinfo &&
-		partition_bound_has_default(partdesc->boundinfo))
-		return partdesc->oids[partdesc->boundinfo->default_index];
+	/*
+	 * As noted above, for non-default, we return list with constant TRUE. If
+	 * the result is NIL during the recursive call for default, it implies
+	 * this is the only other partition which can hold every value of the key
+	 * except NULL. Hence we return the NullTest result skipped earlier.
+	 */
+	if (result == NIL)
+		result = for_default
+			? get_range_nulltest(key)
+			: list_make1(makeBoolConst(true, false));
 
-	return InvalidOid;
+	return result;
 }
 
 /*
@@ -3189,99 +1352,6 @@ get_proposed_default_constraint(List *new_part_constraints)
 }
 
 /*
- * get_partition_bound_num_indexes
- *
- * Returns the number of the entries in the partition bound indexes array.
- */
-static int
-get_partition_bound_num_indexes(PartitionBoundInfo bound)
-{
-	int			num_indexes;
-
-	Assert(bound);
-
-	switch (bound->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-
-			/*
-			 * The number of the entries in the indexes array is same as the
-			 * greatest modulus.
-			 */
-			num_indexes = get_greatest_modulus(bound);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			num_indexes = bound->ndatums;
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			/* Range partitioned table has an extra index. */
-			num_indexes = bound->ndatums + 1;
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) bound->strategy);
-	}
-
-	return num_indexes;
-}
-
-/*
- * get_greatest_modulus
- *
- * Returns the greatest modulus of the hash partition bound. The greatest
- * modulus will be at the end of the datums array because hash partitions are
- * arranged in the ascending order of their modulus and remainders.
- */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
-{
-	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
-	Assert(bound->datums && bound->ndatums > 0);
-	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
-
-	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
-}
-
-/*
- * compute_hash_value
- *
- * Compute the hash value for given not null partition key values.
- */
-static uint64
-compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
-{
-	int			i;
-	int			nkeys = key->partnatts;
-	uint64		rowHash = 0;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-
-	for (i = 0; i < nkeys; i++)
-	{
-		if (!isnull[i])
-		{
-			Datum		hash;
-
-			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
-
-			/*
-			 * Compute hash for each datum value by calling respective
-			 * datatype-specific hash functions of each partition key
-			 * attribute.
-			 */
-			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	return rowHash;
-}
-
-/*
  * satisfies_hash_partition
  *
  * This is an SQL-callable function for use in hash partition constraints.
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 5d3e923cca..b17abb5c7d 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -42,7 +42,6 @@
 #include "access/transam.h"
 #include "access/xact.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_publication.h"
 #include "commands/matview.h"
 #include "commands/trigger.h"
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 4048c3ebc6..cc77ba3701 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -20,6 +20,7 @@
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b586f941a8..8c7caabbc7 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,7 +33,6 @@
 #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"
@@ -49,6 +48,7 @@
 #include "parser/parse_coerce.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
 
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 3bb468bdad..107301ebc7 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -24,7 +24,6 @@
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
-#include "catalog/partition.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_authid.h"
diff --git a/src/backend/utils/cache/Makefile b/src/backend/utils/cache/Makefile
index a943f8ea4b..94511eaf54 100644
--- a/src/backend/utils/cache/Makefile
+++ b/src/backend/utils/cache/Makefile
@@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
-	relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
-	typcache.o ts_cache.o
+OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o partcache.o \
+    relcache.o relmapper.o relfilenodemap.o spccache.o syscache.o \
+    lsyscache.o typcache.o ts_cache.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
new file mode 100644
index 0000000000..5df180b015
--- /dev/null
+++ b/src/backend/utils/cache/partcache.c
@@ -0,0 +1,2114 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.c
+ *		Partitioning related cache data structures and manipulation functions
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *		  src/backend/utils/cache/partcache.c
+ *
+ *-------------------------------------------------------------------------
+*/
+
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "access/nbtree.h"
+#include "access/sysattr.h"
+#include "catalog/partition.h"
+#include "catalog/pg_inherits_fn.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/var.h"
+#include "utils/builtins.h"
+#include "utils/datum.h"
+#include "utils/hashutils.h"
+#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
+#include "utils/ruleutils.h"
+#include "utils/syscache.h"
+
+/*
+ * When qsort'ing partition bounds after reading from the catalog, each bound
+ * is represented with one of the following structs.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int			modulus;
+	int			remainder;
+	int			index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+	int			index;
+	Datum		value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+	int			index;
+	Datum	   *datums;			/* range bound datums */
+	PartitionRangeDatumKind *kind;	/* the kind of each datum */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+static List *generate_partition_qual(Relation rel);
+
+static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
+					 int remainder2);
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
+static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
+							   void *arg);
+static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
+					 List *datums, bool lower);
+static int32 partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2);
+static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
+						   void *arg);
+
+static int partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal);
+static int partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal);
+static int32 partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums);
+static int partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal);
+static int partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder);
+
+static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
+
+/*
+ * RelationBuildPartitionKey
+ *		Build and attach to relcache partition key data of relation
+ *
+ * Partitioning key data is a complex structure; to avoid complicated logic to
+ * free individual elements whenever the relcache entry is flushed, we give it
+ * its own memory context, child of CacheMemoryContext, which can easily be
+ * deleted on its own.  To avoid leaking memory in that context in case of an
+ * error partway through this function, the context is initially created as a
+ * child of CurTransactionContext and only re-parented to CacheMemoryContext
+ * at the end, when no further errors are possible.  Also, we don't make this
+ * context the current context except in very brief code sections, out of fear
+ * that some of our callees allocate memory on their own which would be leaked
+ * permanently.
+ */
+void
+RelationBuildPartitionKey(Relation relation)
+{
+	Form_pg_partitioned_table form;
+	HeapTuple	tuple;
+	bool		isnull;
+	int			i;
+	PartitionKey key;
+	AttrNumber *attrs;
+	oidvector  *opclass;
+	oidvector  *collation;
+	ListCell   *partexprs_item;
+	Datum		datum;
+	MemoryContext partkeycxt,
+				oldcxt;
+	int16		procnum;
+
+	tuple = SearchSysCache1(PARTRELID,
+							ObjectIdGetDatum(RelationGetRelid(relation)));
+
+	/*
+	 * The following happens when we have created our pg_class entry but not
+	 * the pg_partitioned_table entry yet.
+	 */
+	if (!HeapTupleIsValid(tuple))
+		return;
+
+	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
+											   RelationGetRelationName(relation),
+											   MEMCONTEXT_COPY_NAME,
+											   ALLOCSET_SMALL_SIZES);
+
+	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
+												sizeof(PartitionKeyData));
+
+	/* Fixed-length attributes */
+	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	key->strategy = form->partstrat;
+	key->partnatts = form->partnatts;
+
+	/*
+	 * We can rely on the first variable-length attribute being mapped to the
+	 * relevant field of the catalog's C struct, because all previous
+	 * attributes are non-nullable and fixed-length.
+	 */
+	attrs = form->partattrs.values;
+
+	/* But use the hard way to retrieve further variable-length attributes */
+	/* Operator class */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partclass, &isnull);
+	Assert(!isnull);
+	opclass = (oidvector *) DatumGetPointer(datum);
+
+	/* Collation */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partcollation, &isnull);
+	Assert(!isnull);
+	collation = (oidvector *) DatumGetPointer(datum);
+
+	/* Expressions */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partexprs, &isnull);
+	if (!isnull)
+	{
+		char	   *exprString;
+		Node	   *expr;
+
+		exprString = TextDatumGetCString(datum);
+		expr = stringToNode(exprString);
+		pfree(exprString);
+
+		/*
+		 * Run the expressions through const-simplification since the planner
+		 * will be comparing them to similarly-processed qual clause operands,
+		 * and may fail to detect valid matches without this step; fix
+		 * opfuncids while at it.  We don't need to bother with
+		 * canonicalize_qual() though, because partition expressions are not
+		 * full-fledged qualification clauses.
+		 */
+		expr = eval_const_expressions(NULL, expr);
+		fix_opfuncids(expr);
+
+		oldcxt = MemoryContextSwitchTo(partkeycxt);
+		key->partexprs = (List *) copyObject(expr);
+		MemoryContextSwitchTo(oldcxt);
+	}
+
+	oldcxt = MemoryContextSwitchTo(partkeycxt);
+	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
+	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
+
+	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+
+	/* Gather type and collation info as well */
+	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
+	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
+	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
+	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
+	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	MemoryContextSwitchTo(oldcxt);
+
+	/* determine support function number to search for */
+	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
+		HASHEXTENDED_PROC : BTORDER_PROC;
+
+	/* Copy partattrs and fill other per-attribute info */
+	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		AttrNumber	attno = key->partattrs[i];
+		HeapTuple	opclasstup;
+		Form_pg_opclass opclassform;
+		Oid			funcid;
+
+		/* Collect opfamily information */
+		opclasstup = SearchSysCache1(CLAOID,
+									 ObjectIdGetDatum(opclass->values[i]));
+		if (!HeapTupleIsValid(opclasstup))
+			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
+
+		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
+		key->partopfamily[i] = opclassform->opcfamily;
+		key->partopcintype[i] = opclassform->opcintype;
+
+		/* Get a support function for the specified opfamily and datatypes */
+		funcid = get_opfamily_proc(opclassform->opcfamily,
+								   opclassform->opcintype,
+								   opclassform->opcintype,
+								   procnum);
+		if (!OidIsValid(funcid))
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
+							NameStr(opclassform->opcname),
+							(key->strategy == PARTITION_STRATEGY_HASH) ?
+							"hash" : "btree",
+							procnum,
+							format_type_be(opclassform->opcintype))));
+
+		fmgr_info(funcid, &key->partsupfunc[i]);
+
+		/* Collation */
+		key->partcollation[i] = collation->values[i];
+
+		/* Collect type information */
+		if (attno != 0)
+		{
+			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
+
+			key->parttypid[i] = att->atttypid;
+			key->parttypmod[i] = att->atttypmod;
+			key->parttypcoll[i] = att->attcollation;
+		}
+		else
+		{
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			key->parttypid[i] = exprType(lfirst(partexprs_item));
+			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
+			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
+
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(key->parttypid[i],
+							 &key->parttyplen[i],
+							 &key->parttypbyval[i],
+							 &key->parttypalign[i]);
+
+		ReleaseSysCache(opclasstup);
+	}
+
+	ReleaseSysCache(tuple);
+
+	/*
+	 * Success --- reparent our context and make the relcache point to the
+	 * newly constructed key
+	 */
+	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
+	relation->rd_partkeycxt = partkeycxt;
+	relation->rd_partkey = key;
+}
+
+/*
+ * RelationBuildPartitionDesc
+ *		Form rel's partition descriptor
+ *
+ * Not flushed from the cache by RelationClearRelation() unless changed because
+ * of addition or removal of partition.
+ */
+void
+RelationBuildPartitionDesc(Relation rel)
+{
+	List	   *inhoids,
+			   *partoids;
+	Oid		   *oids = NULL;
+	List	   *boundspecs = NIL;
+	ListCell   *cell;
+	int			i,
+				nparts;
+	PartitionKey key = RelationGetPartitionKey(rel);
+	PartitionDesc result;
+	MemoryContext oldcxt;
+
+	int			ndatums = 0;
+	int			default_index = -1;
+
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
+	/* List partitioning specific */
+	PartitionListValue **all_values = NULL;
+	int			null_index = -1;
+
+	/* Range partitioning specific */
+	PartitionRangeBound **rbounds = NULL;
+
+	/*
+	 * The following could happen in situations where rel has a pg_class entry
+	 * but not the pg_partitioned_table entry yet.
+	 */
+	if (key == NULL)
+		return;
+
+	/* Get partition oids from pg_inherits */
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+
+	/* Collect bound spec nodes in a list */
+	i = 0;
+	partoids = NIL;
+	foreach(cell, inhoids)
+	{
+		Oid			inhrelid = lfirst_oid(cell);
+		HeapTuple	tuple;
+		Datum		datum;
+		bool		isnull;
+		Node	   *boundspec;
+
+		tuple = SearchSysCache1(RELOID, inhrelid);
+		if (!HeapTupleIsValid(tuple))
+			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+		/*
+		 * It is possible that the pg_class tuple of a partition has not been
+		 * updated yet to set its relpartbound field.  The only case where
+		 * this happens is when we open the parent relation to check using its
+		 * partition descriptor that a new partition's bound does not overlap
+		 * some existing partition.
+		 */
+		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		{
+			ReleaseSysCache(tuple);
+			continue;
+		}
+
+		datum = SysCacheGetAttr(RELOID, tuple,
+								Anum_pg_class_relpartbound,
+								&isnull);
+		Assert(!isnull);
+		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+		/*
+		 * Sanity check: If the PartitionBoundSpec says this is the default
+		 * partition, its OID should correspond to whatever's stored in
+		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 */
+		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		{
+			Oid			partdefid;
+
+			partdefid = get_default_partition_oid(RelationGetRelid(rel));
+			if (partdefid != inhrelid)
+				elog(ERROR, "expected partdefid %u, but got %u",
+					 inhrelid, partdefid);
+		}
+
+		boundspecs = lappend(boundspecs, boundspec);
+		partoids = lappend_oid(partoids, inhrelid);
+		ReleaseSysCache(tuple);
+	}
+
+	nparts = list_length(partoids);
+
+	if (nparts > 0)
+	{
+		oids = (Oid *) palloc(nparts * sizeof(Oid));
+		i = 0;
+		foreach(cell, partoids)
+			oids[i++] = lfirst_oid(cell);
+
+		/* Convert from node to the internal representation */
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **)
+				palloc(nparts * sizeof(PartitionHashBound *));
+
+			i = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
+		{
+			List	   *non_null_values = NIL;
+
+			/*
+			 * Create a unified list of non-null values across all partitions.
+			 */
+			i = 0;
+			null_index = -1;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				ListCell   *c;
+
+				if (spec->strategy != PARTITION_STRATEGY_LIST)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
+				foreach(c, spec->listdatums)
+				{
+					Const	   *val = castNode(Const, lfirst(c));
+					PartitionListValue *list_value = NULL;
+
+					if (!val->constisnull)
+					{
+						list_value = (PartitionListValue *)
+							palloc0(sizeof(PartitionListValue));
+						list_value->index = i;
+						list_value->value = val->constvalue;
+					}
+					else
+					{
+						/*
+						 * Never put a null into the values array, flag
+						 * instead for the code further down below where we
+						 * construct the actual relcache struct.
+						 */
+						if (null_index != -1)
+							elog(ERROR, "found null more than once");
+						null_index = i;
+					}
+
+					if (list_value)
+						non_null_values = lappend(non_null_values,
+												  list_value);
+				}
+
+				i++;
+			}
+
+			ndatums = list_length(non_null_values);
+
+			/*
+			 * Collect all list values in one array. Alongside the value, we
+			 * also save the index of partition the value comes from.
+			 */
+			all_values = (PartitionListValue **) palloc(ndatums *
+														sizeof(PartitionListValue *));
+			i = 0;
+			foreach(cell, non_null_values)
+			{
+				PartitionListValue *src = lfirst(cell);
+
+				all_values[i] = (PartitionListValue *)
+					palloc(sizeof(PartitionListValue));
+				all_values[i]->value = src->value;
+				all_values[i]->index = src->index;
+				i++;
+			}
+
+			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
+					  qsort_partition_list_value_cmp, (void *) key);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_RANGE)
+		{
+			int			k;
+			PartitionRangeBound **all_bounds,
+					   *prev;
+
+			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
+														  sizeof(PartitionRangeBound *));
+
+			/*
+			 * Create a unified list of range bounds across all the
+			 * partitions.
+			 */
+			i = ndatums = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				PartitionRangeBound *lower,
+						   *upper;
+
+				if (spec->strategy != PARTITION_STRATEGY_RANGE)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the allbounds array
+				 * for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i++;
+					continue;
+				}
+
+				lower = make_one_range_bound(key, i, spec->lowerdatums,
+											 true);
+				upper = make_one_range_bound(key, i, spec->upperdatums,
+											 false);
+				all_bounds[ndatums++] = lower;
+				all_bounds[ndatums++] = upper;
+				i++;
+			}
+
+			Assert(ndatums == nparts * 2 ||
+				   (default_index != -1 && ndatums == (nparts - 1) * 2));
+
+			/* Sort all the bounds in ascending order */
+			qsort_arg(all_bounds, ndatums,
+					  sizeof(PartitionRangeBound *),
+					  qsort_partition_rbound_cmp,
+					  (void *) key);
+
+			/* Save distinct bounds from all_bounds into rbounds. */
+			rbounds = (PartitionRangeBound **)
+				palloc(ndatums * sizeof(PartitionRangeBound *));
+			k = 0;
+			prev = NULL;
+			for (i = 0; i < ndatums; i++)
+			{
+				PartitionRangeBound *cur = all_bounds[i];
+				bool		is_distinct = false;
+				int			j;
+
+				/* Is the current bound distinct from the previous one? */
+				for (j = 0; j < key->partnatts; j++)
+				{
+					Datum		cmpval;
+
+					if (prev == NULL || cur->kind[j] != prev->kind[j])
+					{
+						is_distinct = true;
+						break;
+					}
+
+					/*
+					 * If the bounds are both MINVALUE or MAXVALUE, stop now
+					 * and treat them as equal, since any values after this
+					 * point must be ignored.
+					 */
+					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
+						break;
+
+					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
+											   key->partcollation[j],
+											   cur->datums[j],
+											   prev->datums[j]);
+					if (DatumGetInt32(cmpval) != 0)
+					{
+						is_distinct = true;
+						break;
+					}
+				}
+
+				/*
+				 * Only if the bound is distinct save it into a temporary
+				 * array i.e. rbounds which is later copied into boundinfo
+				 * datums array.
+				 */
+				if (is_distinct)
+					rbounds[k++] = all_bounds[i];
+
+				prev = cur;
+			}
+
+			/* Update ndatums to hold the count of distinct datums. */
+			ndatums = k;
+		}
+		else
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/* Now build the actual relcache partition descriptor */
+	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
+												  RelationGetRelationName(rel),
+												  MEMCONTEXT_COPY_NAME,
+												  ALLOCSET_DEFAULT_SIZES);
+	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+
+	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
+	result->nparts = nparts;
+	if (nparts > 0)
+	{
+		PartitionBoundInfo boundinfo;
+		int		   *mapping;
+		int			next_index = 0;
+
+		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
+
+		boundinfo = (PartitionBoundInfoData *)
+			palloc0(sizeof(PartitionBoundInfoData));
+		boundinfo->strategy = key->strategy;
+		boundinfo->default_index = -1;
+		boundinfo->ndatums = ndatums;
+		boundinfo->null_index = -1;
+		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
+
+		/* Initialize mapping array with invalid values */
+		mapping = (int *) palloc(sizeof(int) * nparts);
+		for (i = 0; i < nparts; i++)
+			mapping[i] = -1;
+
+		switch (key->strategy)
+		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+
+					for (i = 0; i < greatest_modulus; i++)
+						boundinfo->indexes[i] = -1;
+
+					for (i = 0; i < nparts; i++)
+					{
+						int			modulus = hbounds[i]->modulus;
+						int			remainder = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(modulus);
+						boundinfo->datums[i][1] = Int32GetDatum(remainder);
+
+						while (remainder < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[remainder] == -1);
+							boundinfo->indexes[remainder] = i;
+							remainder += modulus;
+						}
+
+						mapping[hbounds[i]->index] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
+			case PARTITION_STRATEGY_LIST:
+				{
+					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
+
+					/*
+					 * Copy values.  Indexes of individual values are mapped
+					 * to canonical values so that they match for any two list
+					 * partitioned tables with same number of partitions and
+					 * same lists per partition.  One way to canonicalize is
+					 * to assign the index in all_values[] of the smallest
+					 * value of each partition, as the index of all of the
+					 * partition's values.
+					 */
+					for (i = 0; i < ndatums; i++)
+					{
+						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
+						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
+															key->parttypbyval[0],
+															key->parttyplen[0]);
+
+						/* If the old index has no mapping, assign one */
+						if (mapping[all_values[i]->index] == -1)
+							mapping[all_values[i]->index] = next_index++;
+
+						boundinfo->indexes[i] = mapping[all_values[i]->index];
+					}
+
+					/*
+					 * If null-accepting partition has no mapped index yet,
+					 * assign one.  This could happen if such partition
+					 * accepts only null and hence not covered in the above
+					 * loop which only handled non-null values.
+					 */
+					if (null_index != -1)
+					{
+						Assert(null_index >= 0);
+						if (mapping[null_index] == -1)
+							mapping[null_index] = next_index++;
+						boundinfo->null_index = mapping[null_index];
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any value not
+						 * specified in the lists of other partitions, hence
+						 * it should not get mapped index while assigning
+						 * those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+
+					/* All partition must now have a valid mapping */
+					Assert(next_index == nparts);
+					break;
+				}
+
+			case PARTITION_STRATEGY_RANGE:
+				{
+					boundinfo->kind = (PartitionRangeDatumKind **)
+						palloc(ndatums *
+							   sizeof(PartitionRangeDatumKind *));
+					boundinfo->indexes = (int *) palloc((ndatums + 1) *
+														sizeof(int));
+
+					for (i = 0; i < ndatums; i++)
+					{
+						int			j;
+
+						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
+																sizeof(Datum));
+						boundinfo->kind[i] = (PartitionRangeDatumKind *)
+							palloc(key->partnatts *
+								   sizeof(PartitionRangeDatumKind));
+						for (j = 0; j < key->partnatts; j++)
+						{
+							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
+								boundinfo->datums[i][j] =
+									datumCopy(rbounds[i]->datums[j],
+											  key->parttypbyval[j],
+											  key->parttyplen[j]);
+							boundinfo->kind[i][j] = rbounds[i]->kind[j];
+						}
+
+						/*
+						 * There is no mapping for invalid indexes.
+						 *
+						 * Any lower bounds in the rbounds array have invalid
+						 * indexes assigned, because the values between the
+						 * previous bound (if there is one) and this (lower)
+						 * bound are not part of the range of any existing
+						 * partition.
+						 */
+						if (rbounds[i]->lower)
+							boundinfo->indexes[i] = -1;
+						else
+						{
+							int			orig_index = rbounds[i]->index;
+
+							/* If the old index has no mapping, assign one */
+							if (mapping[orig_index] == -1)
+								mapping[orig_index] = next_index++;
+
+							boundinfo->indexes[i] = mapping[orig_index];
+						}
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						Assert(default_index >= 0 && mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					boundinfo->indexes[i] = -1;
+					break;
+				}
+
+			default:
+				elog(ERROR, "unexpected partition strategy: %d",
+					 (int) key->strategy);
+		}
+
+		result->boundinfo = boundinfo;
+
+		/*
+		 * Now assign OIDs from the original array into mapped indexes of the
+		 * result array.  Order of OIDs in the former is defined by the
+		 * catalog scan that retrieved them, whereas that in the latter is
+		 * defined by canonicalized representation of the partition bounds.
+		 */
+		for (i = 0; i < nparts; i++)
+			result->oids[mapping[i]] = oids[i];
+		pfree(mapping);
+	}
+
+	MemoryContextSwitchTo(oldcxt);
+	rel->rd_partdesc = result;
+}
+
+/*
+ * Are two partition bound collections logically equal?
+ *
+ * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
+ * This is also useful when b1 and b2 are bound collections of two separate
+ * relations, respectively, because PartitionBoundInfo is a canonical
+ * representation of partition bounds.
+ */
+bool
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
+					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			i;
+
+	if (b1->strategy != b2->strategy)
+		return false;
+
+	if (b1->ndatums != b2->ndatums)
+		return false;
+
+	if (b1->null_index != b2->null_index)
+		return false;
+
+	if (b1->default_index != b2->default_index)
+		return false;
+
+	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	{
+		int			greatest_modulus = get_greatest_modulus(b1);
+
+		/*
+		 * If two hash partitioned tables have different greatest moduli,
+		 * their partition schemes don't match.
+		 */
+		if (greatest_modulus != get_greatest_modulus(b2))
+			return false;
+
+		/*
+		 * We arrange the partitions in the ascending order of their modulus
+		 * and remainders.  Also every modulus is factor of next larger
+		 * modulus.  Therefore we can safely store index of a given partition
+		 * in indexes array at remainder of that partition.  Also entries at
+		 * (remainder + N * modulus) positions in indexes array are all same
+		 * for (modulus, remainder) specification for any partition.  Thus
+		 * datums array from both the given bounds are same, if and only if
+		 * their indexes array will be same.  So, it suffices to compare
+		 * indexes array.
+		 */
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Nonetheless make sure that the bounds are indeed same when the
+		 * indexes match.  Hash partition bound stores modulus and remainder
+		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			Assert((b1->datums[i][0] == b2->datums[i][0] &&
+					b1->datums[i][1] == b2->datums[i][1]));
+#endif
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
+		{
+			int			j;
+
+			for (j = 0; j < partnatts; j++)
+			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->kind != NULL)
+				{
+					/* The different kinds of bound all differ from each other */
+					if (b1->kind[i][j] != b2->kind[i][j])
+						return false;
+
+					/*
+					 * Non-finite bounds are equal without further
+					 * examination.
+					 */
+					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
+						continue;
+				}
+
+				/*
+				 * Compare the actual values. Note that it would be both
+				 * incorrect and unsafe to invoke the comparison operator
+				 * derived from the partitioning specification here.  It would
+				 * be incorrect because we want the relcache entry to be
+				 * updated for ANY change to the partition bounds, not just
+				 * those that the partitioning operator thinks are
+				 * significant.  It would be unsafe because we might reach
+				 * this code in the context of an aborted transaction, and an
+				 * arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be
+				 * safe.
+				 */
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  parttypbyval[j], parttyplen[j]))
+					return false;
+			}
+
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+		}
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
+			return false;
+	}
+	return true;
+}
+
+/*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specification.
+ */
+PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo dest;
+	int			i;
+	int			ndatums;
+	int			partnatts;
+	int			num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	num_indexes = get_partition_bound_num_indexes(src);
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+														 sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+															   sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int			j;
+
+		/*
+		 * For a corresponding to hash partition, datums array will have two
+		 * elements - modulus and remainder.
+		 */
+		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
+		int			natts = hash_part ? 2 : partnatts;
+
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+
+		for (j = 0; j < natts; j++)
+		{
+			bool		byval;
+			int			typlen;
+
+			if (hash_part)
+			{
+				typlen = sizeof(int32); /* Always int4 */
+				byval = true;	/* int4 is pass-by-value */
+			}
+			else
+			{
+				byval = key->parttypbyval[j];
+				typlen = key->parttyplen[j];
+			}
+
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   byval, typlen);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
+ * check_new_partition_bound
+ *
+ * Checks if the new partition's bound overlaps any of the existing partitions
+ * of parent.  Also performs additional checks as necessary per strategy.
+ */
+void
+check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
+	ParseState *pstate = make_parsestate(NULL);
+	int			with = -1;
+	bool		overlap = false;
+
+	if (spec->is_default)
+	{
+		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+			return;
+
+		/* Default partition already exists, error out. */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+				 parser_errposition(pstate, spec->location)));
+	}
+
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					Datum	  **datums = boundinfo->datums;
+					int			ndatums = boundinfo->ndatums;
+					int			greatest_modulus;
+					int			remainder;
+					int			offset;
+					bool		valid_modulus = true;
+					int			prev_modulus,	/* Previous largest modulus */
+								next_modulus;	/* Next largest modulus */
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.
+					 *
+					 * Get the greatest (modulus, remainder) pair contained in
+					 * boundinfo->datums that is less than or equal to the
+					 * (spec->modulus, spec->remainder) pair.
+					 */
+					offset = partition_hash_bsearch(key, boundinfo,
+													spec->modulus,
+													spec->remainder);
+					if (offset < 0)
+					{
+						next_modulus = DatumGetInt32(datums[0][0]);
+						valid_modulus = (next_modulus % spec->modulus) == 0;
+					}
+					else
+					{
+						prev_modulus = DatumGetInt32(datums[offset][0]);
+						valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+						if (valid_modulus && (offset + 1) < ndatums)
+						{
+							next_modulus = DatumGetInt32(datums[offset + 1][0]);
+							valid_modulus = (next_modulus % spec->modulus) == 0;
+						}
+					}
+
+					if (!valid_modulus)
+						ereport(ERROR,
+								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
+
+					greatest_modulus = get_greatest_modulus(boundinfo);
+					remainder = spec->remainder;
+
+					/*
+					 * Normally, the lowest remainder that could conflict with
+					 * the new partition is equal to the remainder specified
+					 * for the new partition, but when the new partition has a
+					 * modulus higher than any used so far, we need to adjust.
+					 */
+					if (remainder >= greatest_modulus)
+						remainder = remainder % greatest_modulus;
+
+					/* Check every potentially-conflicting remainder. */
+					do
+					{
+						if (boundinfo->indexes[remainder] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[remainder];
+							break;
+						}
+						remainder += spec->modulus;
+					} while (remainder < greatest_modulus);
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_LIST:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+
+				if (partdesc->nparts > 0)
+				{
+					ListCell   *cell;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					foreach(cell, spec->listdatums)
+					{
+						Const	   *val = castNode(Const, lfirst(cell));
+
+						if (!val->constisnull)
+						{
+							int			offset;
+							bool		equal;
+
+							offset = partition_list_bsearch(key, boundinfo,
+															val->constvalue,
+															&equal);
+							if (offset >= 0 && equal)
+							{
+								overlap = true;
+								with = boundinfo->indexes[offset];
+								break;
+							}
+						}
+						else if (partition_bound_accepts_nulls(boundinfo))
+						{
+							overlap = true;
+							with = boundinfo->null_index;
+							break;
+						}
+					}
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				PartitionRangeBound *lower,
+						   *upper;
+
+				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
+				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
+
+				/*
+				 * First check if the resulting range would be empty with
+				 * specified lower and upper bounds
+				 */
+				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
+										 upper) >= 0)
+				{
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("empty range bound specified for partition \"%s\"",
+									relname),
+							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
+									   get_range_partbound_string(spec->lowerdatums),
+									   get_range_partbound_string(spec->upperdatums)),
+							 parser_errposition(pstate, spec->location)));
+				}
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					int			offset;
+					bool		equal;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Test whether the new lower bound (which is treated
+					 * inclusively as part of the new partition) lies inside
+					 * an existing partition, or in a gap.
+					 *
+					 * If it's inside an existing partition, the bound at
+					 * offset + 1 will be the upper bound of that partition,
+					 * and its index will be >= 0.
+					 *
+					 * If it's in a gap, the bound at offset + 1 will be the
+					 * lower bound of the next partition, and its index will
+					 * be -1. This is also true if there is no next partition,
+					 * since the index array is initialised with an extra -1
+					 * at the end.
+					 */
+					offset = partition_range_bsearch(key, boundinfo, lower,
+													 &equal);
+
+					if (boundinfo->indexes[offset + 1] < 0)
+					{
+						/*
+						 * Check that the new partition will fit in the gap.
+						 * For it to fit, the new upper bound must be less
+						 * than or equal to the lower bound of the next
+						 * partition, if there is one.
+						 */
+						if (offset + 1 < boundinfo->ndatums)
+						{
+							int32		cmpval;
+							Datum 	   *datums;
+							PartitionRangeDatumKind *kind;
+							bool		is_lower;
+
+							datums = boundinfo->datums[offset + 1];
+							kind = boundinfo->kind[offset + 1];
+							is_lower = (boundinfo->indexes[offset + 1] == -1);
+
+							cmpval = partition_rbound_cmp(key, datums, kind,
+														  is_lower, upper);
+							if (cmpval < 0)
+							{
+								/*
+								 * The new partition overlaps with the
+								 * existing partition between offset + 1 and
+								 * offset + 2.
+								 */
+								overlap = true;
+								with = boundinfo->indexes[offset + 2];
+							}
+						}
+					}
+					else
+					{
+						/*
+						 * The new partition overlaps with the existing
+						 * partition between offset and offset + 1.
+						 */
+						overlap = true;
+						with = boundinfo->indexes[offset + 1];
+					}
+				}
+
+				break;
+			}
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	if (overlap)
+	{
+		Assert(with >= 0);
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" would overlap partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[with])),
+				 parser_errposition(pstate, spec->location)));
+	}
+}
+
+/*
+ * RelationGetPartitionQual
+ *
+ * Returns a list of partition quals
+ */
+List *
+RelationGetPartitionQual(Relation rel)
+{
+	/* Quick exit */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
+	return generate_partition_qual(rel);
+}
+
+/*
+ * get_partition_qual_relid
+ *
+ * Returns an expression tree describing the passed-in relation's partition
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
+ */
+Expr *
+get_partition_qual_relid(Oid relid)
+{
+	Relation	rel = heap_open(relid, AccessShareLock);
+	Expr	   *result = NULL;
+	List	   *and_args;
+
+	/* Do the work only if this relation is a partition. */
+	if (rel->rd_rel->relispartition)
+	{
+		and_args = generate_partition_qual(rel);
+
+		if (and_args == NIL)
+			result = NULL;
+		else if (list_length(and_args) > 1)
+			result = makeBoolExpr(AND_EXPR, and_args, -1);
+		else
+			result = linitial(and_args);
+	}
+
+	/* Keep the lock. */
+	heap_close(rel, NoLock);
+
+	return result;
+}
+
+/*
+ * Checks if any of the 'attnums' is a partition key attribute for rel
+ *
+ * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
+ * partition key expression.  It's possible for a column to be both used
+ * directly and as part of an expression; if that happens, *used_in_expr may
+ * end up as either true or false.  That's OK for current uses of this
+ * function, because *used_in_expr is only used to tailor the error message
+ * text.
+ */
+bool
+has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr)
+{
+	PartitionKey key;
+	int			partnatts;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	int			i;
+
+	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		return false;
+
+	key = RelationGetPartitionKey(rel);
+	partnatts = get_partition_natts(key);
+	partexprs = get_partition_exprs(key);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+		{
+			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
+							  attnums))
+			{
+				if (used_in_expr)
+					*used_in_expr = false;
+				return true;
+			}
+		}
+		else
+		{
+			/* Arbitrary expression */
+			Node	   *expr = (Node *) lfirst(partexprs_item);
+			Bitmapset  *expr_attrs = NULL;
+
+			/* Find all attributes referenced */
+			pull_varattnos(expr, 1, &expr_attrs);
+			partexprs_item = lnext(partexprs_item);
+
+			if (bms_overlap(attnums, expr_attrs))
+			{
+				if (used_in_expr)
+					*used_in_expr = true;
+				return true;
+			}
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple
+ *		Finds partition of relation which accepts the partition key specified
+ *		in values and isnull
+ *
+ * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
+ * found or -1 if none found.
+ */
+int
+get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
+{
+	int			bound_offset;
+	int			part_index = -1;
+	PartitionKey key = RelationGetPartitionKey(relation);
+	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
+
+	/* Route as appropriate based on partitioning strategy. */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundInfo boundinfo = partdesc->boundinfo;
+				int			greatest_modulus = get_greatest_modulus(boundinfo);
+				uint64		rowHash = compute_hash_value(key, values, isnull);
+
+				part_index = boundinfo->indexes[rowHash % greatest_modulus];
+			}
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			if (isnull[0])
+			{
+				if (partition_bound_accepts_nulls(partdesc->boundinfo))
+					part_index = partdesc->boundinfo->null_index;
+			}
+			else
+			{
+				bool		equal = false;
+
+				bound_offset = partition_list_bsearch(key,
+													  partdesc->boundinfo,
+													  values[0], &equal);
+				if (bound_offset >= 0 && equal)
+					part_index = partdesc->boundinfo->indexes[bound_offset];
+			}
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				bool		equal = false,
+							range_partkey_has_null = false;
+				int			i;
+
+				/*
+				 * No range includes NULL, so this will be accepted by the
+				 * default partition if there is one, and otherwise rejected.
+				 */
+				for (i = 0; i < key->partnatts; i++)
+				{
+					if (isnull[i])
+					{
+						range_partkey_has_null = true;
+						break;
+					}
+				}
+
+				if (!range_partkey_has_null)
+				{
+					bound_offset = partition_range_datum_bsearch(key,
+														partdesc->boundinfo,
+														key->partnatts,
+														values,
+														&equal);
+					/*
+					 * The bound at bound_offset is less than or equal to the
+					 * tuple value, so the bound at offset+1 is the upper
+					 * bound of the partition we're looking for, if there
+					 * actually exists one.
+					 */
+					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
+				}
+			}
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/*
+	 * part_index < 0 means we failed to find a partition of this parent. Use
+	 * the default partition, if there is one.
+	 */
+	if (part_index < 0)
+		part_index = partdesc->boundinfo->default_index;
+
+	return part_index;
+}
+
+/*
+ * get_greatest_modulus
+ *
+ * Returns the greatest modulus of the hash partition bound. The greatest
+ * modulus will be at the end of the datums array because hash partitions are
+ * arranged in the ascending order of their modulus and remainders.
+ */
+int
+get_greatest_modulus(PartitionBoundInfo bound)
+{
+	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
+	Assert(bound->datums && bound->ndatums > 0);
+	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
+
+	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+uint64
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+	int			i;
+	int			nkeys = key->partnatts;
+	uint64		rowHash = 0;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+	for (i = 0; i < nkeys; i++)
+	{
+		if (!isnull[i])
+		{
+			Datum		hash;
+
+			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
+
+			/*
+			 * Compute hash for each datum value by calling respective
+			 * datatype-specific hash functions of each partition key
+			 * attribute.
+			 */
+			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+	if (partdesc && partdesc->boundinfo &&
+		partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
+
+/* Module-local functions. */
+
+/*
+ * generate_partition_qual
+ *
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
+ *
+ * Result expression tree is stored CacheMemoryContext to ensure it survives
+ * as long as the relcache entry. But we should be running in a less long-lived
+ * working context. To avoid leaking cache memory if this routine fails partway
+ * through, we build in working memory and then copy the completed structure
+ * into cache memory.
+ */
+static List *
+generate_partition_qual(Relation rel)
+{
+	HeapTuple	tuple;
+	MemoryContext oldcxt;
+	Datum		boundDatum;
+	bool		isnull;
+	PartitionBoundSpec *bound;
+	List	   *my_qual = NIL,
+			   *result = NIL;
+	Relation	parent;
+	bool		found_whole_row;
+
+	/* Guard against stack overflow due to overly deep partition tree */
+	check_stack_depth();
+
+	/* Quick copy */
+	if (rel->rd_partcheck != NIL)
+		return copyObject(rel->rd_partcheck);
+
+	/* Grab at least an AccessShareLock on the parent table */
+	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
+					   AccessShareLock);
+
+	/* Get pg_class.relpartbound */
+	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	boundDatum = SysCacheGetAttr(RELOID, tuple,
+								 Anum_pg_class_relpartbound,
+								 &isnull);
+	if (isnull)					/* should not happen */
+		elog(ERROR, "relation \"%s\" has relpartbound = null",
+			 RelationGetRelationName(rel));
+	bound = castNode(PartitionBoundSpec,
+					 stringToNode(TextDatumGetCString(boundDatum)));
+	ReleaseSysCache(tuple);
+
+	my_qual = get_qual_from_partbound(rel, parent, bound);
+
+	/* 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;
+
+	/*
+	 * Change Vars to have partition's attnos instead of the parent's. We do
+	 * this after we concatenate the parent's quals, because we want every Var
+	 * in it to bear this relation's attnos. It's safe to assume varno = 1
+	 * here.
+	 */
+	result = map_partition_varattnos(result, 1, rel, parent,
+									 &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");
+
+	/* Save a copy in the relcache */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	rel->rd_partcheck = copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* Keep the parent locked until commit */
+	heap_close(parent, NoLock);
+
+	return result;
+}
+
+/*
+ * partition_hbound_cmp
+ *
+ * Compares modulus first, then remainder if modulus are equal.
+ */
+static int32
+partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
+{
+	if (modulus1 < modulus2)
+		return -1;
+	if (modulus1 > modulus2)
+		return 1;
+	if (modulus1 == modulus2 && remainder1 != remainder2)
+		return (remainder1 > remainder2) ? 1 : -1;
+	return 0;
+}
+
+/*
+ * qsort_partition_hbound_cmp
+ *
+ * We sort hash bounds by modulus, then by remainder.
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+
+	return partition_hbound_cmp(h1->modulus, h1->remainder,
+								h2->modulus, h2->remainder);
+}
+
+/*
+ * qsort_partition_list_value_cmp
+ *
+ * Compare two list partition bound datums
+ */
+static int32
+qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+{
+	Datum		val1 = (*(const PartitionListValue **) a)->value,
+				val2 = (*(const PartitionListValue **) b)->value;
+	PartitionKey key = (PartitionKey) arg;
+
+	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+										   key->partcollation[0],
+										   val1, val2));
+}
+
+/*
+ * make_one_range_bound
+ *
+ * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
+ * and a flag telling whether the bound is lower or not.  Made into a function
+ * because there are multiple sites that want to use this facility.
+ */
+static PartitionRangeBound *
+make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
+{
+	PartitionRangeBound *bound;
+	ListCell   *lc;
+	int			i;
+
+	Assert(datums != NIL);
+
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
+	bound->index = index;
+	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
+	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
+													  sizeof(PartitionRangeDatumKind));
+	bound->lower = lower;
+
+	i = 0;
+	foreach(lc, datums)
+	{
+		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+
+		/* What's contained in this range datum? */
+		bound->kind[i] = datum->kind;
+
+		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		{
+			Const	   *val = castNode(Const, datum->value);
+
+			if (val->constisnull)
+				elog(ERROR, "invalid range bound datum");
+			bound->datums[i] = val->constvalue;
+		}
+
+		i++;
+	}
+
+	return bound;
+}
+
+/*
+ * partition_rbound_cmp
+ *
+ * Return for two range bounds whether the 1st one (specified in datums1,
+ * kind1, and lower1) is <, =, or > the bound specified in *b2.
+ *
+ * Note that if the values of the two range bounds compare equal, then we take
+ * into account whether they are upper or lower bounds, and an upper bound is
+ * considered to be smaller than a lower bound. This is important to the way
+ * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
+ * structure, which only stores the upper bound of a common boundary between
+ * two contiguous partitions.
+ */
+static int32
+partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2)
+{
+	int32		cmpval = 0;		/* placate compiler */
+	int			i;
+	Datum	   *datums2 = b2->datums;
+	PartitionRangeDatumKind *kind2 = b2->kind;
+	bool		lower2 = b2->lower;
+
+	for (i = 0; i < key->partnatts; i++)
+	{
+		/*
+		 * First, handle cases where the column is unbounded, which should not
+		 * invoke the comparison procedure, and should not consider any later
+		 * columns. Note that the PartitionRangeDatumKind enum elements
+		 * compare the same way as the values they represent.
+		 */
+		if (kind1[i] < kind2[i])
+			return -1;
+		else if (kind1[i] > kind2[i])
+			return 1;
+		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+
+			/*
+			 * The column bounds are both MINVALUE or both MAXVALUE. No later
+			 * columns should be considered, but we still need to compare
+			 * whether they are upper or lower bounds.
+			 */
+			break;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 datums1[i],
+												 datums2[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	/*
+	 * If the comparison is anything other than equal, we're done. If they
+	 * compare equal though, we still have to consider whether the boundaries
+	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
+	 * two.
+	 */
+	if (cmpval == 0 && lower1 != lower2)
+		cmpval = lower1 ? 1 : -1;
+
+	return cmpval;
+}
+
+/* Used when sorting range bounds across all range partitions */
+static int32
+qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
+{
+	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
+	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
+	PartitionKey key = (PartitionKey) arg;
+
+	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+}
+
+/*
+ * partition_list_bsearch
+ *		Returns the index of the greatest bound datum that is less than equal
+ * 		to the given value or -1 if all of the bound datums are greater
+ *
+ * *is_equal is set to true if the bound datum at the returned index is equal
+ * to the input value.
+ */
+static int
+partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+												 key->partcollation[0],
+												 boundinfo->datums[mid][0],
+												 value));
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_rbound_datum_cmp
+ *
+ * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
+ * is <, =, or > partition key of tuple (tuple_datums)
+ */
+static int32
+partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums)
+{
+	int			i;
+	int32		cmpval = -1;
+
+	for (i = 0; i < n_tuple_datums; i++)
+	{
+		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
+			return -1;
+		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
+			return 1;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 rb_datums[i],
+												 tuple_datums[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	return cmpval;
+}
+
+/*
+ * partition_range_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given range bound or -1 if all of the range bounds are
+ *		greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input range bound
+ */
+static int
+partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_cmp(key,
+									  boundinfo->datums[mid],
+									  boundinfo->kind[mid],
+									  (boundinfo->indexes[mid] == -1),
+									  probe);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_range_datum_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given tuple or -1 if all of the range bounds are greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input tuple.
+ */
+static int
+partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_datum_cmp(key,
+											boundinfo->datums[mid],
+											boundinfo->kind[mid],
+											values,
+											nvalues);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_hash_bsearch
+ *		Returns the index of the greatest (modulus, remainder) pair that is
+ *		less than or equal to the given (modulus, remainder) pair or -1 if
+ *		all of them are greater
+ */
+static int
+partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval,
+					bound_modulus,
+					bound_remainder;
+
+		mid = (lo + hi + 1) / 2;
+		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
+		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
+		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
+									  modulus, remainder);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+
+			if (cmpval == 0)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * get_partition_bound_num_indexes
+ *
+ * Returns the number of the entries in the partition bound indexes array.
+ */
+static int
+get_partition_bound_num_indexes(PartitionBoundInfo bound)
+{
+	int			num_indexes;
+
+	Assert(bound);
+
+	switch (bound->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * The number of the entries in the indexes array is same as the
+			 * greatest modulus.
+			 */
+			num_indexes = get_greatest_modulus(bound);
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			num_indexes = bound->ndatums;
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			/* Range partitioned table has an extra index. */
+			num_indexes = bound->ndatums + 1;
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) bound->strategy);
+	}
+
+	return num_indexes;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 1ebf9c4ed2..a747f53e7f 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -81,6 +81,7 @@
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partcache.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -261,7 +262,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
 static Relation AllocateRelationDesc(Form_pg_class relp);
 static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
 static void RelationBuildTupleDesc(Relation relation);
-static void RelationBuildPartitionKey(Relation relation);
 static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
 static void RelationInitPhysicalAddr(Relation relation);
 static void load_critical_index(Oid indexoid, Oid heapoid);
@@ -809,209 +809,6 @@ RelationBuildRuleLock(Relation relation)
 }
 
 /*
- * RelationBuildPartitionKey
- *		Build and attach to relcache partition key data of relation
- *
- * Partitioning key data is a complex structure; to avoid complicated logic to
- * free individual elements whenever the relcache entry is flushed, we give it
- * its own memory context, child of CacheMemoryContext, which can easily be
- * deleted on its own.  To avoid leaking memory in that context in case of an
- * error partway through this function, the context is initially created as a
- * child of CurTransactionContext and only re-parented to CacheMemoryContext
- * at the end, when no further errors are possible.  Also, we don't make this
- * context the current context except in very brief code sections, out of fear
- * that some of our callees allocate memory on their own which would be leaked
- * permanently.
- */
-static void
-RelationBuildPartitionKey(Relation relation)
-{
-	Form_pg_partitioned_table form;
-	HeapTuple	tuple;
-	bool		isnull;
-	int			i;
-	PartitionKey key;
-	AttrNumber *attrs;
-	oidvector  *opclass;
-	oidvector  *collation;
-	ListCell   *partexprs_item;
-	Datum		datum;
-	MemoryContext partkeycxt,
-				oldcxt;
-	int16		procnum;
-
-	tuple = SearchSysCache1(PARTRELID,
-							ObjectIdGetDatum(RelationGetRelid(relation)));
-
-	/*
-	 * The following happens when we have created our pg_class entry but not
-	 * the pg_partitioned_table entry yet.
-	 */
-	if (!HeapTupleIsValid(tuple))
-		return;
-
-	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
-											   RelationGetRelationName(relation),
-											   MEMCONTEXT_COPY_NAME,
-											   ALLOCSET_SMALL_SIZES);
-
-	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
-												sizeof(PartitionKeyData));
-
-	/* Fixed-length attributes */
-	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	key->strategy = form->partstrat;
-	key->partnatts = form->partnatts;
-
-	/*
-	 * We can rely on the first variable-length attribute being mapped to the
-	 * relevant field of the catalog's C struct, because all previous
-	 * attributes are non-nullable and fixed-length.
-	 */
-	attrs = form->partattrs.values;
-
-	/* But use the hard way to retrieve further variable-length attributes */
-	/* Operator class */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partclass, &isnull);
-	Assert(!isnull);
-	opclass = (oidvector *) DatumGetPointer(datum);
-
-	/* Collation */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partcollation, &isnull);
-	Assert(!isnull);
-	collation = (oidvector *) DatumGetPointer(datum);
-
-	/* Expressions */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partexprs, &isnull);
-	if (!isnull)
-	{
-		char	   *exprString;
-		Node	   *expr;
-
-		exprString = TextDatumGetCString(datum);
-		expr = stringToNode(exprString);
-		pfree(exprString);
-
-		/*
-		 * Run the expressions through const-simplification since the planner
-		 * will be comparing them to similarly-processed qual clause operands,
-		 * and may fail to detect valid matches without this step; fix
-		 * opfuncids while at it.  We don't need to bother with
-		 * canonicalize_qual() though, because partition expressions are not
-		 * full-fledged qualification clauses.
-		 */
-		expr = eval_const_expressions(NULL, expr);
-		fix_opfuncids(expr);
-
-		oldcxt = MemoryContextSwitchTo(partkeycxt);
-		key->partexprs = (List *) copyObject(expr);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
-	oldcxt = MemoryContextSwitchTo(partkeycxt);
-	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
-	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
-
-	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-
-	/* Gather type and collation info as well */
-	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
-	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
-	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
-	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
-	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	MemoryContextSwitchTo(oldcxt);
-
-	/* determine support function number to search for */
-	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
-		HASHEXTENDED_PROC : BTORDER_PROC;
-
-	/* Copy partattrs and fill other per-attribute info */
-	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		AttrNumber	attno = key->partattrs[i];
-		HeapTuple	opclasstup;
-		Form_pg_opclass opclassform;
-		Oid			funcid;
-
-		/* Collect opfamily information */
-		opclasstup = SearchSysCache1(CLAOID,
-									 ObjectIdGetDatum(opclass->values[i]));
-		if (!HeapTupleIsValid(opclasstup))
-			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
-
-		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
-		key->partopfamily[i] = opclassform->opcfamily;
-		key->partopcintype[i] = opclassform->opcintype;
-
-		/* Get a support function for the specified opfamily and datatypes */
-		funcid = get_opfamily_proc(opclassform->opcfamily,
-								   opclassform->opcintype,
-								   opclassform->opcintype,
-								   procnum);
-		if (!OidIsValid(funcid))
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
-							NameStr(opclassform->opcname),
-							(key->strategy == PARTITION_STRATEGY_HASH) ?
-							"hash" : "btree",
-							procnum,
-							format_type_be(opclassform->opcintype))));
-
-		fmgr_info(funcid, &key->partsupfunc[i]);
-
-		/* Collation */
-		key->partcollation[i] = collation->values[i];
-
-		/* Collect type information */
-		if (attno != 0)
-		{
-			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
-
-			key->parttypid[i] = att->atttypid;
-			key->parttypmod[i] = att->atttypmod;
-			key->parttypcoll[i] = att->attcollation;
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			key->parttypid[i] = exprType(lfirst(partexprs_item));
-			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
-			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
-
-			partexprs_item = lnext(partexprs_item);
-		}
-		get_typlenbyvalalign(key->parttypid[i],
-							 &key->parttyplen[i],
-							 &key->parttypbyval[i],
-							 &key->parttypalign[i]);
-
-		ReleaseSysCache(opclasstup);
-	}
-
-	ReleaseSysCache(tuple);
-
-	/*
-	 * Success --- reparent our context and make the relcache point to the
-	 * newly constructed key
-	 */
-	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
-	relation->rd_partkeycxt = partkeycxt;
-	relation->rd_partkey = key;
-}
-
-/*
  *		equalRuleLocks
  *
  *		Determine whether two RuleLocks are equivalent
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2faf0ca26e..894c8f4091 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -14,63 +14,22 @@
 #define PARTITION_H
 
 #include "fmgr.h"
-#include "executor/tuptable.h"
-#include "nodes/execnodes.h"
-#include "parser/parse_node.h"
 #include "utils/rel.h"
 
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-/*
- * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
- * associated with partitioned tables as part of its partition descriptor.
- *
- * The internal structure is opaque outside partition.c.
- */
-typedef struct PartitionBoundInfoData *PartitionBoundInfo;
-
-/*
- * Information about partitions of a partitioned table.
- */
-typedef struct PartitionDescData
-{
-	int			nparts;			/* Number of partitions */
-	Oid		   *oids;			/* OIDs of partitions */
-	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
-} PartitionDescData;
-
-typedef struct PartitionDescData *PartitionDesc;
-
-extern void RelationBuildPartitionDesc(Relation relation);
-extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
-					   bool *parttypbyval, PartitionBoundInfo b1,
-					   PartitionBoundInfo b2);
-extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key);
-
-extern void check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec);
 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 fromrel_varno,
 						Relation to_rel, Relation from_rel,
 						bool *found_whole_row);
-extern List *RelationGetPartitionQual(Relation rel);
-extern Expr *get_partition_qual_relid(Oid relid);
-extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr);
 
-extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
 extern Oid	get_default_partition_oid(Oid parentId);
 extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
-extern int get_partition_for_tuple(Relation relation, Datum *values,
-						bool *isnull);
-
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 3df9c498bb..c53dfcc265 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -13,10 +13,10 @@
 #ifndef EXECPARTITION_H
 #define EXECPARTITION_H
 
-#include "catalog/partition.h"
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "utils/partcache.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
diff --git a/src/include/utils/partcache.h b/src/include/utils/partcache.h
new file mode 100644
index 0000000000..5d4caeda3a
--- /dev/null
+++ b/src/include/utils/partcache.h
@@ -0,0 +1,191 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.h
+ *		Header file for partitioning related cached data structures and
+ *		manipulation functions
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partcache.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTCACHE_H
+#define PARTCACHE_H
+
+#include "postgres.h"
+
+#include "fmgr.h"
+#include "nodes/parsenodes.h"
+#include "utils/lsyscache.h"
+#include "utils/relcache.h"
+
+/*
+ * Information about the partition key of a relation
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partitioning strategy */
+	int16		partnatts;		/* number of columns in the partition key */
+	AttrNumber *partattrs;		/* attribute numbers of columns in the
+								 * partition key */
+	List	   *partexprs;		/* list of expressions in the partitioning
+								 * key, or NIL */
+
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+
+	/* Partitioning collation per attribute */
+	Oid		   *partcollation;
+
+	/* Type information per attribute */
+	Oid		   *parttypid;
+	int32	   *parttypmod;
+	int16	   *parttyplen;
+	bool	   *parttypbyval;
+	char	   *parttypalign;
+	Oid		   *parttypcoll;
+}			PartitionKeyData;
+
+typedef struct PartitionKeyData *PartitionKey;
+
+typedef struct PartitionBoundInfoData *PartitionBoundInfo;
+
+/*
+ * Information about partitions of a partitioned table.
+ */
+typedef struct PartitionDescData
+{
+	int			nparts;			/* Number of partitions */
+	Oid		   *oids;			/* OIDs of partitions */
+	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
+} PartitionDescData;
+
+typedef struct PartitionDescData *PartitionDesc;
+
+/*
+ * Information about bounds of a partitioned relation
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+	char		strategy;		/* hash, list or range? */
+	int			ndatums;		/* Length of the datums following array */
+	Datum	  **datums;
+	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+									 * NULL for hash and list partitioned
+									 * tables */
+	int		   *indexes;		/* Partition indexes */
+	int			null_index;		/* Index of the null-accepting partition; -1
+								 * if there isn't one */
+	int			default_index;	/* Index of the default partition; -1 if there
+								 * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * PartitionKey inquiry functions
+ */
+static inline int
+get_partition_strategy(PartitionKey key)
+{
+	return key->strategy;
+}
+
+static inline int
+get_partition_natts(PartitionKey key)
+{
+	return key->partnatts;
+}
+
+static inline List *
+get_partition_exprs(PartitionKey key)
+{
+	return key->partexprs;
+}
+
+/*
+ * PartitionKey inquiry functions - one column
+ */
+static inline int16
+get_partition_col_attnum(PartitionKey key, int col)
+{
+	return key->partattrs[col];
+}
+
+static inline Oid
+get_partition_col_typid(PartitionKey key, int col)
+{
+	return key->parttypid[col];
+}
+
+static inline int32
+get_partition_col_typmod(PartitionKey key, int col)
+{
+	return key->parttypmod[col];
+}
+
+extern void RelationBuildPartitionKey(Relation relation);
+extern void RelationBuildPartitionDesc(Relation relation);
+extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
+					   bool *parttypbyval, PartitionBoundInfo b1,
+					   PartitionBoundInfo b2);
+
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
+
+extern void check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec);
+
+extern List *RelationGetPartitionQual(Relation rel);
+extern Expr *get_partition_qual_relid(Oid relid);
+
+extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr);
+
+extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+
+extern int get_greatest_modulus(PartitionBoundInfo b);
+extern uint64 compute_hash_value(PartitionKey key, Datum *values,
+						bool *isnull);
+
+/* For tuple routing */
+extern int get_partition_for_tuple(Relation relation, Datum *values,
+						bool *isnull);
+
+#endif							/* PARTCACHE_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index aa8add544a..b531ef0121 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -24,6 +24,7 @@
 #include "rewrite/prs2lock.h"
 #include "storage/block.h"
 #include "storage/relfilenode.h"
+#include "utils/partcache.h"
 #include "utils/relcache.h"
 #include "utils/reltrigger.h"
 
@@ -47,36 +48,6 @@ typedef struct LockInfoData
 typedef LockInfoData *LockInfo;
 
 /*
- * Information about the partition key of a relation
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partitioning strategy */
-	int16		partnatts;		/* number of columns in the partition key */
-	AttrNumber *partattrs;		/* attribute numbers of columns in the
-								 * partition key */
-	List	   *partexprs;		/* list of expressions in the partitioning
-								 * key, or NIL */
-
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-
-	/* Partitioning collation per attribute */
-	Oid		   *partcollation;
-
-	/* Type information per attribute */
-	Oid		   *parttypid;
-	int32	   *parttypmod;
-	int16	   *parttyplen;
-	bool	   *parttypbyval;
-	char	   *parttypalign;
-	Oid		   *parttypcoll;
-}			PartitionKeyData;
-
-typedef struct PartitionKeyData *PartitionKey;
-
-/*
  * Here are the contents of a relation cache entry.
  */
 
@@ -593,48 +564,6 @@ typedef struct ViewOptions
 #define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
 
 /*
- * PartitionKey inquiry functions
- */
-static inline int
-get_partition_strategy(PartitionKey key)
-{
-	return key->strategy;
-}
-
-static inline int
-get_partition_natts(PartitionKey key)
-{
-	return key->partnatts;
-}
-
-static inline List *
-get_partition_exprs(PartitionKey key)
-{
-	return key->partexprs;
-}
-
-/*
- * PartitionKey inquiry functions - one column
- */
-static inline int16
-get_partition_col_attnum(PartitionKey key, int col)
-{
-	return key->partattrs[col];
-}
-
-static inline Oid
-get_partition_col_typid(PartitionKey key, int col)
-{
-	return key->parttypid[col];
-}
-
-static inline int32
-get_partition_col_typmod(PartitionKey key, int col)
-{
-	return key->parttypmod[col];
-}
-
-/*
  * RelationGetPartitionDesc
  *		Returns partition descriptor for a relation.
  */
-- 
2.11.0

#2Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#1)
Re: reorganizing partitioning code (was: Re: [HACKERS] path toward faster partition pruning)

On 2018/02/13 23:08, Ashutosh Bapat wrote:

On Tue, Feb 13, 2018 at 2:17 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Agree with the proposed reorganizing and adding a partcache.c, which I
tried to do in the attached patch.

* The new src/backend/utils/cache/partcache.c contains functions that
initialize relcache's partitioning related fields. Various partition
bound comparison and search functions (and then some) that work off of the
cached information are moved.

Are you moving partition bound comparison functions to partcache.c?
They will also used by optimizer, so may be leave them out of
partcache.c?

Yes, I moved the partition bound comparison and search functions, because
I thought that they should live with the other code that manages the
cached information. So, I moved not only the code that reads the catalogs
and builds the partition key, partition (bound) descriptor, and partition
qual (for individual partitions), but also the code that uses those
structures.

So, with the new arrangement, optimizer will include utils/partcache.h,
instead of catalog/partition.h.

Thanks,
Amit

#3Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#1)
1 attachment(s)
Re: reorganizing partitioning code (was: Re: [HACKERS] path toward faster partition pruning)

On 2018/02/14 10:00, Amit Langote wrote:

Agree with the proposed reorganizing and adding a partcache.c, which I
tried to do in the attached patch.

* The new src/backend/utils/cache/partcache.c contains functions that
initialize relcache's partitioning related fields. Various partition
bound comparison and search functions (and then some) that work off of the
cached information are moved. Also, since we cache partition qual,
interface functions RelationGetPartitioQual(Relation) and
get_partition_qual_relid(Oid) are moved too.

* The new src/include/utils/partcache.h contains various struct
definitions that are moved from backend/catalog/partition.c,
include/catalog/partition.h, and include/utils/rel.h. Also, declarations
of interface functions of partcache.c.

Attached updated version, where I removed #include "catalog/partition.h"
from even more places and also moved map_partition_varattnos() next to
map_variable_attnos() in rewriteManip.c.

Now, after applying the patch -

Files #including partition.h

File Line
0 src/backend/catalog/heap.c 44 #include "catalog/partition.h"
1 src/backend/catalog/partition.c 22 #include "catalog/partition.h"
2 src/backend/commands/indexcmds.c 26 #include "catalog/partition.h"
3 src/backend/commands/tablecmds.c 33 #include "catalog/partition.h"
4 src/backend/utils/cache/partcache.c 23 #include "catalog/partition.h"

Files #including partcache.h

File Line
0 src/backend/optimizer/path/joinrels.c 24 #include "utils/partcache.h"
1 src/backend/optimizer/prep/prepunion.c 51 #include "utils/partcache.h"
2 src/backend/optimizer/util/relnode.c 30 #include "utils/partcache.h"
3 src/backend/utils/cache/partcache.c 37 #include "utils/partcache.h"
4 src/backend/utils/cache/relcache.c 83 #include "utils/partcache.h"
5 src/include/executor/execPartition.h 19 #include "utils/partcache.h"
6 src/include/utils/rel.h 27 #include "utils/partcache.h"

Thanks,
Amit

Attachments:

v2-0001-Reorganize-partitioning-code.patchtext/plain; charset=UTF-8; name=v2-0001-Reorganize-partitioning-code.patchDownload
From e8eddecb8571d4c273f30d9c80961fa932404b73 Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Tue, 13 Feb 2018 15:59:30 +0900
Subject: [PATCH v2] Reorganize partitioning code

---
 src/backend/catalog/partition.c         | 3879 ++++++++-----------------------
 src/backend/executor/execExpr.c         |    1 +
 src/backend/executor/execMain.c         |    1 -
 src/backend/executor/execPartition.c    |    1 +
 src/backend/executor/nodeModifyTable.c  |    1 +
 src/backend/libpq/hba.c                 |    1 +
 src/backend/optimizer/path/joinrels.c   |    2 +-
 src/backend/optimizer/prep/prepunion.c  |    2 +-
 src/backend/optimizer/util/plancat.c    |    1 -
 src/backend/optimizer/util/relnode.c    |    2 +-
 src/backend/rewrite/rewriteManip.c      |   45 +
 src/backend/tcop/utility.c              |    1 +
 src/backend/utils/adt/ruleutils.c       |    1 -
 src/backend/utils/cache/Makefile        |    6 +-
 src/backend/utils/cache/partcache.c     | 2115 +++++++++++++++++
 src/backend/utils/cache/relcache.c      |  206 +-
 src/backend/utils/misc/pg_controldata.c |    1 +
 src/include/catalog/partition.h         |   44 -
 src/include/commands/tablecmds.h        |    1 -
 src/include/executor/execPartition.h    |    2 +-
 src/include/executor/executor.h         |    1 -
 src/include/rewrite/rewriteManip.h      |    4 +
 src/include/utils/partcache.h           |  191 ++
 src/include/utils/rel.h                 |   73 +-
 24 files changed, 3322 insertions(+), 3260 deletions(-)
 create mode 100644 src/backend/utils/cache/partcache.c
 create mode 100644 src/include/utils/partcache.h

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 31c80c7f1a..8bac2a75ae 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,11 +15,7 @@
 
 #include "postgres.h"
 
-#include "access/hash.h"
-#include "access/heapam.h"
 #include "access/htup_details.h"
-#include "access/nbtree.h"
-#include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
@@ -52,98 +48,9 @@
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
-#include "utils/ruleutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-	char		strategy;		/* hash, list or range? */
-	int			ndatums;		/* Length of the datums following array */
-	Datum	  **datums;
-	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-									 * NULL for hash and list partitioned
-									 * tables */
-	int		   *indexes;		/* Partition indexes */
-	int			null_index;		/* Index of the null-accepting partition; -1
-								 * if there isn't one */
-	int			default_index;	/* Index of the default partition; -1 if there
-								 * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-	int			modulus;
-	int			remainder;
-	int			index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-	int			index;
-	Datum		value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-	int			index;
-	Datum	   *datums;			/* range bound datums */
-	PartitionRangeDatumKind *kind;	/* the kind of each datum */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
-static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
-static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
-							   void *arg);
-static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
-						   void *arg);
-
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
@@ -159,2948 +66,1159 @@ static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 				   bool for_default);
 static List *get_range_nulltest(PartitionKey key);
-static List *generate_partition_qual(Relation rel);
-
-static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
-					 List *datums, bool lower);
-static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
-					 int remainder2);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal);
-static int partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder);
-
-static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int	get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 
 /* SQL-callable function for use in hash partition CHECK constraints */
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 /*
- * RelationBuildPartitionDesc
- *		Form rel's partition descriptor
+ * check_default_allows_bound
  *
- * Not flushed from the cache by RelationClearRelation() unless changed because
- * of addition or removal of partition.
+ * This function checks if there exists a row in the default partition that
+ * would properly belong to the new partition being added.  If it finds one,
+ * it throws an error.
  */
 void
-RelationBuildPartitionDesc(Relation rel)
+check_default_allows_bound(Relation parent, Relation default_rel,
+						   PartitionBoundSpec *new_spec)
 {
-	List	   *inhoids,
-			   *partoids;
-	Oid		   *oids = NULL;
-	List	   *boundspecs = NIL;
-	ListCell   *cell;
-	int			i,
-				nparts;
-	PartitionKey key = RelationGetPartitionKey(rel);
-	PartitionDesc result;
-	MemoryContext oldcxt;
-
-	int			ndatums = 0;
-	int			default_index = -1;
-
-	/* Hash partitioning specific */
-	PartitionHashBound **hbounds = NULL;
-
-	/* List partitioning specific */
-	PartitionListValue **all_values = NULL;
-	int			null_index = -1;
+	List	   *new_part_constraints;
+	List	   *def_part_constraints;
+	List	   *all_parts;
+	ListCell   *lc;
 
-	/* Range partitioning specific */
-	PartitionRangeBound **rbounds = NULL;
+	new_part_constraints = (new_spec->strategy == PARTITION_STRATEGY_LIST)
+		? get_qual_for_list(parent, new_spec)
+		: get_qual_for_range(parent, new_spec, false);
+	def_part_constraints =
+		get_proposed_default_constraint(new_part_constraints);
 
 	/*
-	 * The following could happen in situations where rel has a pg_class entry
-	 * but not the pg_partitioned_table entry yet.
+	 * If the existing constraints on the default partition imply that it will
+	 * not contain any row that would belong to the new partition, we can
+	 * avoid scanning the default partition.
 	 */
-	if (key == NULL)
+	if (PartConstraintImpliedByRelConstraint(default_rel, def_part_constraints))
+	{
+		ereport(INFO,
+				(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
+						RelationGetRelationName(default_rel))));
 		return;
+	}
 
-	/* Get partition oids from pg_inherits */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+	/*
+	 * Scan the default partition and its subpartitions, and check for rows
+	 * that do not satisfy the revised partition constraints.
+	 */
+	if (default_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+		all_parts = find_all_inheritors(RelationGetRelid(default_rel),
+										AccessExclusiveLock, NULL);
+	else
+		all_parts = list_make1_oid(RelationGetRelid(default_rel));
 
-	/* Collect bound spec nodes in a list */
-	i = 0;
-	partoids = NIL;
-	foreach(cell, inhoids)
+	foreach(lc, all_parts)
 	{
-		Oid			inhrelid = lfirst_oid(cell);
+		Oid			part_relid = lfirst_oid(lc);
+		Relation	part_rel;
+		Expr	   *constr;
+		Expr	   *partition_constraint;
+		EState	   *estate;
 		HeapTuple	tuple;
-		Datum		datum;
-		bool		isnull;
-		Node	   *boundspec;
-
-		tuple = SearchSysCache1(RELOID, inhrelid);
-		if (!HeapTupleIsValid(tuple))
-			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+		ExprState  *partqualstate = NULL;
+		Snapshot	snapshot;
+		TupleDesc	tupdesc;
+		ExprContext *econtext;
+		HeapScanDesc scan;
+		MemoryContext oldCxt;
+		TupleTableSlot *tupslot;
 
-		/*
-		 * It is possible that the pg_class tuple of a partition has not been
-		 * updated yet to set its relpartbound field.  The only case where
-		 * this happens is when we open the parent relation to check using its
-		 * partition descriptor that a new partition's bound does not overlap
-		 * some existing partition.
-		 */
-		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		/* Lock already taken above. */
+		if (part_relid != RelationGetRelid(default_rel))
 		{
-			ReleaseSysCache(tuple);
-			continue;
-		}
+			part_rel = heap_open(part_relid, NoLock);
+
+			/*
+			 * If the partition constraints on default partition child imply
+			 * that it will not contain any row that would belong to the new
+			 * partition, we can avoid scanning the child table.
+			 */
+			if (PartConstraintImpliedByRelConstraint(part_rel,
+													 def_part_constraints))
+			{
+				ereport(INFO,
+						(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
+								RelationGetRelationName(part_rel))));
 
-		datum = SysCacheGetAttr(RELOID, tuple,
-								Anum_pg_class_relpartbound,
-								&isnull);
-		Assert(!isnull);
-		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+				heap_close(part_rel, NoLock);
+				continue;
+			}
+		}
+		else
+			part_rel = default_rel;
 
 		/*
-		 * Sanity check: If the PartitionBoundSpec says this is the default
-		 * partition, its OID should correspond to whatever's stored in
-		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 * Only RELKIND_RELATION relations (i.e. leaf partitions) need to be
+		 * scanned.
 		 */
-		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		if (part_rel->rd_rel->relkind != RELKIND_RELATION)
 		{
-			Oid			partdefid;
+			if (part_rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+				ereport(WARNING,
+						(errcode(ERRCODE_CHECK_VIOLATION),
+						 errmsg("skipped scanning foreign table \"%s\" which is a partition of default partition \"%s\"",
+								RelationGetRelationName(part_rel),
+								RelationGetRelationName(default_rel))));
+
+			if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
+				heap_close(part_rel, NoLock);
 
-			partdefid = get_default_partition_oid(RelationGetRelid(rel));
-			if (partdefid != inhrelid)
-				elog(ERROR, "expected partdefid %u, but got %u",
-					 inhrelid, partdefid);
+			continue;
 		}
 
-		boundspecs = lappend(boundspecs, boundspec);
-		partoids = lappend_oid(partoids, inhrelid);
-		ReleaseSysCache(tuple);
-	}
+		tupdesc = CreateTupleDescCopy(RelationGetDescr(part_rel));
+		constr = linitial(def_part_constraints);
+		partition_constraint = (Expr *)
+			map_partition_varattnos((List *) constr,
+									1, part_rel, parent, NULL);
+		estate = CreateExecutorState();
 
-	nparts = list_length(partoids);
+		/* Build expression execution states for partition check quals */
+		partqualstate = ExecPrepareExpr(partition_constraint, estate);
 
-	if (nparts > 0)
-	{
-		oids = (Oid *) palloc(nparts * sizeof(Oid));
-		i = 0;
-		foreach(cell, partoids)
-			oids[i++] = lfirst_oid(cell);
+		econtext = GetPerTupleExprContext(estate);
+		snapshot = RegisterSnapshot(GetLatestSnapshot());
+		scan = heap_beginscan(part_rel, snapshot, 0, NULL);
+		tupslot = MakeSingleTupleTableSlot(tupdesc);
 
-		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_HASH)
+		/*
+		 * Switch to per-tuple memory context and reset it for each tuple
+		 * produced, so we don't leak memory.
+		 */
+		oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
+
+		while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
 		{
-			ndatums = nparts;
-			hbounds = (PartitionHashBound **)
-				palloc(nparts * sizeof(PartitionHashBound *));
+			ExecStoreTuple(tuple, tupslot, InvalidBuffer, false);
+			econtext->ecxt_scantuple = tupslot;
 
-			i = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
+			if (!ExecCheck(partqualstate, econtext))
+				ereport(ERROR,
+						(errcode(ERRCODE_CHECK_VIOLATION),
+						 errmsg("updated partition constraint for default partition \"%s\" would be violated by some row",
+								RelationGetRelationName(default_rel))));
 
-				if (spec->strategy != PARTITION_STRATEGY_HASH)
-					elog(ERROR, "invalid strategy in partition bound spec");
+			ResetExprContext(econtext);
+			CHECK_FOR_INTERRUPTS();
+		}
 
-				hbounds[i] = (PartitionHashBound *)
-					palloc(sizeof(PartitionHashBound));
+		MemoryContextSwitchTo(oldCxt);
+		heap_endscan(scan);
+		UnregisterSnapshot(snapshot);
+		ExecDropSingleTupleTableSlot(tupslot);
+		FreeExecutorState(estate);
 
-				hbounds[i]->modulus = spec->modulus;
-				hbounds[i]->remainder = spec->remainder;
-				hbounds[i]->index = i;
-				i++;
-			}
+		if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
+			heap_close(part_rel, NoLock);	/* keep the lock until commit */
+	}
+}
 
-			/* Sort all the bounds in ascending order */
-			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
-				  qsort_partition_hbound_cmp);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_LIST)
-		{
-			List	   *non_null_values = NIL;
+/*
+ * get_partition_parent
+ *
+ * Returns inheritance parent of a partition by scanning pg_inherits
+ *
+ * Note: Because this function assumes that the relation whose OID is passed
+ * as an argument will have precisely one parent, it should only be called
+ * when it is known that the relation is a partition.
+ */
+Oid
+get_partition_parent(Oid relid)
+{
+	Form_pg_inherits form;
+	Relation	catalogRelation;
+	SysScanDesc scan;
+	ScanKeyData key[2];
+	HeapTuple	tuple;
+	Oid			result;
 
-			/*
-			 * Create a unified list of non-null values across all partitions.
-			 */
-			i = 0;
-			null_index = -1;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				ListCell   *c;
+	catalogRelation = heap_open(InheritsRelationId, AccessShareLock);
 
-				if (spec->strategy != PARTITION_STRATEGY_LIST)
-					elog(ERROR, "invalid strategy in partition bound spec");
+	ScanKeyInit(&key[0],
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	ScanKeyInit(&key[1],
+				Anum_pg_inherits_inhseqno,
+				BTEqualStrategyNumber, F_INT4EQ,
+				Int32GetDatum(1));
 
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the list of non-null
-				 * datums for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i;
-					i++;
-					continue;
-				}
+	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId, true,
+							  NULL, 2, key);
 
-				foreach(c, spec->listdatums)
-				{
-					Const	   *val = castNode(Const, lfirst(c));
-					PartitionListValue *list_value = NULL;
+	tuple = systable_getnext(scan);
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "could not find tuple for parent of relation %u", relid);
 
-					if (!val->constisnull)
-					{
-						list_value = (PartitionListValue *)
-							palloc0(sizeof(PartitionListValue));
-						list_value->index = i;
-						list_value->value = val->constvalue;
-					}
-					else
-					{
-						/*
-						 * Never put a null into the values array, flag
-						 * instead for the code further down below where we
-						 * construct the actual relcache struct.
-						 */
-						if (null_index != -1)
-							elog(ERROR, "found null more than once");
-						null_index = i;
-					}
+	form = (Form_pg_inherits) GETSTRUCT(tuple);
+	result = form->inhparent;
 
-					if (list_value)
-						non_null_values = lappend(non_null_values,
-												  list_value);
-				}
+	systable_endscan(scan);
+	heap_close(catalogRelation, AccessShareLock);
 
-				i++;
-			}
+	return result;
+}
 
-			ndatums = list_length(non_null_values);
+/*
+ * get_qual_from_partbound
+ *		Given a parser node for partition bound, return the list of executable
+ *		expressions as partition constraint
+ */
+List *
+get_qual_from_partbound(Relation rel, Relation parent,
+						PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	List	   *my_qual = NIL;
 
-			/*
-			 * Collect all list values in one array. Alongside the value, we
-			 * also save the index of partition the value comes from.
-			 */
-			all_values = (PartitionListValue **) palloc(ndatums *
-														sizeof(PartitionListValue *));
-			i = 0;
-			foreach(cell, non_null_values)
-			{
-				PartitionListValue *src = lfirst(cell);
+	Assert(key != NULL);
 
-				all_values[i] = (PartitionListValue *)
-					palloc(sizeof(PartitionListValue));
-				all_values[i]->value = src->value;
-				all_values[i]->index = src->index;
-				i++;
-			}
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+			my_qual = get_qual_for_hash(parent, spec);
+			break;
 
-			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
-					  qsort_partition_list_value_cmp, (void *) key);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_RANGE)
-		{
-			int			k;
-			PartitionRangeBound **all_bounds,
-					   *prev;
+		case PARTITION_STRATEGY_LIST:
+			Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+			my_qual = get_qual_for_list(parent, spec);
+			break;
 
-			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
-														  sizeof(PartitionRangeBound *));
+		case PARTITION_STRATEGY_RANGE:
+			Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+			my_qual = get_qual_for_range(parent, spec, false);
+			break;
 
-			/*
-			 * Create a unified list of range bounds across all the
-			 * partitions.
-			 */
-			i = ndatums = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				PartitionRangeBound *lower,
-						   *upper;
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
 
-				if (spec->strategy != PARTITION_STRATEGY_RANGE)
-					elog(ERROR, "invalid strategy in partition bound spec");
+	return my_qual;
+}
 
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the allbounds array
-				 * for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i++;
-					continue;
-				}
+/* Module-local functions */
 
-				lower = make_one_range_bound(key, i, spec->lowerdatums,
-											 true);
-				upper = make_one_range_bound(key, i, spec->upperdatums,
-											 false);
-				all_bounds[ndatums++] = lower;
-				all_bounds[ndatums++] = upper;
-				i++;
-			}
-
-			Assert(ndatums == nparts * 2 ||
-				   (default_index != -1 && ndatums == (nparts - 1) * 2));
-
-			/* Sort all the bounds in ascending order */
-			qsort_arg(all_bounds, ndatums,
-					  sizeof(PartitionRangeBound *),
-					  qsort_partition_rbound_cmp,
-					  (void *) key);
-
-			/* Save distinct bounds from all_bounds into rbounds. */
-			rbounds = (PartitionRangeBound **)
-				palloc(ndatums * sizeof(PartitionRangeBound *));
-			k = 0;
-			prev = NULL;
-			for (i = 0; i < ndatums; i++)
-			{
-				PartitionRangeBound *cur = all_bounds[i];
-				bool		is_distinct = false;
-				int			j;
-
-				/* Is the current bound distinct from the previous one? */
-				for (j = 0; j < key->partnatts; j++)
-				{
-					Datum		cmpval;
-
-					if (prev == NULL || cur->kind[j] != prev->kind[j])
-					{
-						is_distinct = true;
-						break;
-					}
-
-					/*
-					 * If the bounds are both MINVALUE or MAXVALUE, stop now
-					 * and treat them as equal, since any values after this
-					 * point must be ignored.
-					 */
-					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
-						break;
-
-					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
-											   key->partcollation[j],
-											   cur->datums[j],
-											   prev->datums[j]);
-					if (DatumGetInt32(cmpval) != 0)
-					{
-						is_distinct = true;
-						break;
-					}
-				}
-
-				/*
-				 * Only if the bound is distinct save it into a temporary
-				 * array i.e. rbounds which is later copied into boundinfo
-				 * datums array.
-				 */
-				if (is_distinct)
-					rbounds[k++] = all_bounds[i];
+/*
+ * get_partition_operator
+ *
+ * Return oid of the operator of given strategy for a given partition key
+ * column.
+ */
+static Oid
+get_partition_operator(PartitionKey key, int col, StrategyNumber strategy,
+					   bool *need_relabel)
+{
+	Oid			operoid;
 
-				prev = cur;
-			}
+	/*
+	 * First check if there exists an operator of the given strategy, with
+	 * this column's type as both its lefttype and righttype, in the
+	 * partitioning operator family specified for the column.
+	 */
+	operoid = get_opfamily_member(key->partopfamily[col],
+								  key->parttypid[col],
+								  key->parttypid[col],
+								  strategy);
 
-			/* Update ndatums to hold the count of distinct datums. */
-			ndatums = k;
-		}
-		else
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
+	/*
+	 * If one doesn't exist, we must resort to using an operator in the same
+	 * operator family but with the operator class declared input type.  It is
+	 * OK to do so, because the column's type is known to be binary-coercible
+	 * with the operator class input type (otherwise, the operator class in
+	 * question would not have been accepted as the partitioning operator
+	 * class).  We must however inform the caller to wrap the non-Const
+	 * expression with a RelabelType node to denote the implicit coercion. It
+	 * ensures that the resulting expression structurally matches similarly
+	 * processed expressions within the optimizer.
+	 */
+	if (!OidIsValid(operoid))
+	{
+		operoid = get_opfamily_member(key->partopfamily[col],
+									  key->partopcintype[col],
+									  key->partopcintype[col],
+									  strategy);
+		if (!OidIsValid(operoid))
+			elog(ERROR, "missing operator %d(%u,%u) in opfamily %u",
+				 strategy, key->partopcintype[col], key->partopcintype[col],
+				 key->partopfamily[col]);
+		*need_relabel = true;
 	}
+	else
+		*need_relabel = false;
 
-	/* Now build the actual relcache partition descriptor */
-	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
-												  RelationGetRelationName(rel),
-												  MEMCONTEXT_COPY_NAME,
-												  ALLOCSET_DEFAULT_SIZES);
-	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+	return operoid;
+}
 
-	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
-	result->nparts = nparts;
-	if (nparts > 0)
-	{
-		PartitionBoundInfo boundinfo;
-		int		   *mapping;
-		int			next_index = 0;
-
-		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
-
-		boundinfo = (PartitionBoundInfoData *)
-			palloc0(sizeof(PartitionBoundInfoData));
-		boundinfo->strategy = key->strategy;
-		boundinfo->default_index = -1;
-		boundinfo->ndatums = ndatums;
-		boundinfo->null_index = -1;
-		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
-
-		/* Initialize mapping array with invalid values */
-		mapping = (int *) palloc(sizeof(int) * nparts);
-		for (i = 0; i < nparts; i++)
-			mapping[i] = -1;
+/*
+ * make_partition_op_expr
+ *		Returns an Expr for the given partition key column with arg1 and
+ *		arg2 as its leftop and rightop, respectively
+ */
+static Expr *
+make_partition_op_expr(PartitionKey key, int keynum,
+					   uint16 strategy, Expr *arg1, Expr *arg2)
+{
+	Oid			operoid;
+	bool		need_relabel = false;
+	Expr	   *result = NULL;
 
-		switch (key->strategy)
-		{
-			case PARTITION_STRATEGY_HASH:
-				{
-					/* Modulus are stored in ascending order */
-					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+	/* Get the correct btree operator for this partitioning column */
+	operoid = get_partition_operator(key, keynum, strategy, &need_relabel);
 
-					boundinfo->indexes = (int *) palloc(greatest_modulus *
-														sizeof(int));
+	/*
+	 * Chosen operator may be such that the non-Const operand needs to be
+	 * coerced, so apply the same; see the comment in
+	 * get_partition_operator().
+	 */
+	if (!IsA(arg1, Const) &&
+		(need_relabel ||
+		 key->partcollation[keynum] != key->parttypcoll[keynum]))
+		arg1 = (Expr *) makeRelabelType(arg1,
+										key->partopcintype[keynum],
+										-1,
+										key->partcollation[keynum],
+										COERCE_EXPLICIT_CAST);
 
-					for (i = 0; i < greatest_modulus; i++)
-						boundinfo->indexes[i] = -1;
+	/* Generate the actual expression */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			{
+				List	   *elems = (List *) arg2;
+				int			nelems = list_length(elems);
 
-					for (i = 0; i < nparts; i++)
-					{
-						int			modulus = hbounds[i]->modulus;
-						int			remainder = hbounds[i]->remainder;
-
-						boundinfo->datums[i] = (Datum *) palloc(2 *
-																sizeof(Datum));
-						boundinfo->datums[i][0] = Int32GetDatum(modulus);
-						boundinfo->datums[i][1] = Int32GetDatum(remainder);
-
-						while (remainder < greatest_modulus)
-						{
-							/* overlap? */
-							Assert(boundinfo->indexes[remainder] == -1);
-							boundinfo->indexes[remainder] = i;
-							remainder += modulus;
-						}
-
-						mapping[hbounds[i]->index] = i;
-						pfree(hbounds[i]);
-					}
-					pfree(hbounds);
-					break;
-				}
+				Assert(nelems >= 1);
+				Assert(keynum == 0);
 
-			case PARTITION_STRATEGY_LIST:
+				if (nelems > 1 &&
+					!type_is_array(key->parttypid[keynum]))
 				{
-					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
-
-					/*
-					 * Copy values.  Indexes of individual values are mapped
-					 * to canonical values so that they match for any two list
-					 * partitioned tables with same number of partitions and
-					 * same lists per partition.  One way to canonicalize is
-					 * to assign the index in all_values[] of the smallest
-					 * value of each partition, as the index of all of the
-					 * partition's values.
-					 */
-					for (i = 0; i < ndatums; i++)
-					{
-						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
-						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
-															key->parttypbyval[0],
-															key->parttyplen[0]);
-
-						/* If the old index has no mapping, assign one */
-						if (mapping[all_values[i]->index] == -1)
-							mapping[all_values[i]->index] = next_index++;
-
-						boundinfo->indexes[i] = mapping[all_values[i]->index];
-					}
+					ArrayExpr  *arrexpr;
+					ScalarArrayOpExpr *saopexpr;
 
-					/*
-					 * If null-accepting partition has no mapped index yet,
-					 * assign one.  This could happen if such partition
-					 * accepts only null and hence not covered in the above
-					 * loop which only handled non-null values.
-					 */
-					if (null_index != -1)
-					{
-						Assert(null_index >= 0);
-						if (mapping[null_index] == -1)
-							mapping[null_index] = next_index++;
-						boundinfo->null_index = mapping[null_index];
-					}
+					/* Construct an ArrayExpr for the right-hand inputs */
+					arrexpr = makeNode(ArrayExpr);
+					arrexpr->array_typeid =
+									get_array_type(key->parttypid[keynum]);
+					arrexpr->array_collid = key->parttypcoll[keynum];
+					arrexpr->element_typeid = key->parttypid[keynum];
+					arrexpr->elements = elems;
+					arrexpr->multidims = false;
+					arrexpr->location = -1;
 
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						/*
-						 * The default partition accepts any value not
-						 * specified in the lists of other partitions, hence
-						 * it should not get mapped index while assigning
-						 * those for non-null datums.
-						 */
-						Assert(default_index >= 0 &&
-							   mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
+					/* Build leftop = ANY (rightop) */
+					saopexpr = makeNode(ScalarArrayOpExpr);
+					saopexpr->opno = operoid;
+					saopexpr->opfuncid = get_opcode(operoid);
+					saopexpr->useOr = true;
+					saopexpr->inputcollid = key->partcollation[keynum];
+					saopexpr->args = list_make2(arg1, arrexpr);
+					saopexpr->location = -1;
 
-					/* All partition must now have a valid mapping */
-					Assert(next_index == nparts);
-					break;
+					result = (Expr *) saopexpr;
 				}
-
-			case PARTITION_STRATEGY_RANGE:
+				else
 				{
-					boundinfo->kind = (PartitionRangeDatumKind **)
-						palloc(ndatums *
-							   sizeof(PartitionRangeDatumKind *));
-					boundinfo->indexes = (int *) palloc((ndatums + 1) *
-														sizeof(int));
+					List	   *elemops = NIL;
+					ListCell   *lc;
 
-					for (i = 0; i < ndatums; i++)
+					foreach (lc, elems)
 					{
-						int			j;
-
-						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
-																sizeof(Datum));
-						boundinfo->kind[i] = (PartitionRangeDatumKind *)
-							palloc(key->partnatts *
-								   sizeof(PartitionRangeDatumKind));
-						for (j = 0; j < key->partnatts; j++)
-						{
-							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
-								boundinfo->datums[i][j] =
-									datumCopy(rbounds[i]->datums[j],
-											  key->parttypbyval[j],
-											  key->parttyplen[j]);
-							boundinfo->kind[i][j] = rbounds[i]->kind[j];
-						}
-
-						/*
-						 * There is no mapping for invalid indexes.
-						 *
-						 * Any lower bounds in the rbounds array have invalid
-						 * indexes assigned, because the values between the
-						 * previous bound (if there is one) and this (lower)
-						 * bound are not part of the range of any existing
-						 * partition.
-						 */
-						if (rbounds[i]->lower)
-							boundinfo->indexes[i] = -1;
-						else
-						{
-							int			orig_index = rbounds[i]->index;
-
-							/* If the old index has no mapping, assign one */
-							if (mapping[orig_index] == -1)
-								mapping[orig_index] = next_index++;
-
-							boundinfo->indexes[i] = mapping[orig_index];
-						}
-					}
+						Expr   *elem = lfirst(lc),
+							   *elemop;
 
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						Assert(default_index >= 0 && mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
+						elemop = make_opclause(operoid,
+											   BOOLOID,
+											   false,
+											   arg1, elem,
+											   InvalidOid,
+											   key->partcollation[keynum]);
+						elemops = lappend(elemops, elemop);
 					}
-					boundinfo->indexes[i] = -1;
-					break;
-				}
 
-			default:
-				elog(ERROR, "unexpected partition strategy: %d",
-					 (int) key->strategy);
-		}
+					result = nelems > 1 ? makeBoolExpr(OR_EXPR, elemops, -1) : linitial(elemops);
+				}
+				break;
+			}
 
-		result->boundinfo = boundinfo;
+		case PARTITION_STRATEGY_RANGE:
+			result = make_opclause(operoid,
+								   BOOLOID,
+								   false,
+								   arg1, arg2,
+								   InvalidOid,
+								   key->partcollation[keynum]);
+			break;
 
-		/*
-		 * Now assign OIDs from the original array into mapped indexes of the
-		 * result array.  Order of OIDs in the former is defined by the
-		 * catalog scan that retrieved them, whereas that in the latter is
-		 * defined by canonicalized representation of the partition bounds.
-		 */
-		for (i = 0; i < nparts; i++)
-			result->oids[mapping[i]] = oids[i];
-		pfree(mapping);
+		default:
+			elog(ERROR, "invalid partitioning strategy");
+			break;
 	}
 
-	MemoryContextSwitchTo(oldcxt);
-	rel->rd_partdesc = result;
+	return result;
 }
 
 /*
- * Are two partition bound collections logically equal?
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder corresponding to a
+ * partition, this function returns CHECK constraint expression Node for that
+ * partition.
  *
- * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
- * This is also useful when b1 and b2 are bound collections of two separate
- * relations, respectively, because PartitionBoundInfo is a canonical
- * representation of partition bounds.
+ * The partition constraint for a hash partition is always a call to the
+ * built-in function satisfies_hash_partition().  The first two arguments are
+ * the modulus and remainder for the partition; the remaining arguments are the
+ * values to be hashed.
  */
-bool
-partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
-					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
 {
+	PartitionKey key = RelationGetPartitionKey(parent);
+	FuncExpr   *fexpr;
+	Node	   *relidConst;
+	Node	   *modulusConst;
+	Node	   *remainderConst;
+	List	   *args;
+	ListCell   *partexprs_item;
 	int			i;
 
-	if (b1->strategy != b2->strategy)
-		return false;
+	/* Fixed arguments. */
+	relidConst = (Node *) makeConst(OIDOID,
+									-1,
+									InvalidOid,
+									sizeof(Oid),
+									ObjectIdGetDatum(RelationGetRelid(parent)),
+									false,
+									true);
 
-	if (b1->ndatums != b2->ndatums)
-		return false;
+	modulusConst = (Node *) makeConst(INT4OID,
+									  -1,
+									  InvalidOid,
+									  sizeof(int32),
+									  Int32GetDatum(spec->modulus),
+									  false,
+									  true);
 
-	if (b1->null_index != b2->null_index)
-		return false;
+	remainderConst = (Node *) makeConst(INT4OID,
+										-1,
+										InvalidOid,
+										sizeof(int32),
+										Int32GetDatum(spec->remainder),
+										false,
+										true);
 
-	if (b1->default_index != b2->default_index)
-		return false;
+	args = list_make3(relidConst, modulusConst, remainderConst);
+	partexprs_item = list_head(key->partexprs);
 
-	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	/* Add an argument for each key column. */
+	for (i = 0; i < key->partnatts; i++)
 	{
-		int			greatest_modulus = get_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_greatest_modulus(b2))
-			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their modulus
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
+		Node	   *keyCol;
 
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
-	}
-	else
-	{
-		for (i = 0; i < b1->ndatums; i++)
+		/* Left operand */
+		if (key->partattrs[i] != 0)
 		{
-			int			j;
-
-			for (j = 0; j < partnatts; j++)
-			{
-				/* For range partitions, the bounds might not be finite. */
-				if (b1->kind != NULL)
-				{
-					/* The different kinds of bound all differ from each other */
-					if (b1->kind[i][j] != b2->kind[i][j])
-						return false;
-
-					/*
-					 * Non-finite bounds are equal without further
-					 * examination.
-					 */
-					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
-						continue;
-				}
-
-				/*
-				 * Compare the actual values. Note that it would be both
-				 * incorrect and unsafe to invoke the comparison operator
-				 * derived from the partitioning specification here.  It would
-				 * be incorrect because we want the relcache entry to be
-				 * updated for ANY change to the partition bounds, not just
-				 * those that the partitioning operator thinks are
-				 * significant.  It would be unsafe because we might reach
-				 * this code in the context of an aborted transaction, and an
-				 * arbitrary partitioning operator might not be safe in that
-				 * context.  datumIsEqual() should be simple enough to be
-				 * safe.
-				 */
-				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-								  parttypbyval[j], parttyplen[j]))
-					return false;
-			}
-
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
+			keyCol = (Node *) makeVar(1,
+									  key->partattrs[i],
+									  key->parttypid[i],
+									  key->parttypmod[i],
+									  key->parttypcoll[i],
+									  0);
+		}
+		else
+		{
+			keyCol = (Node *) copyObject(lfirst(partexprs_item));
+			partexprs_item = lnext(partexprs_item);
 		}
 
-		/* There are ndatums+1 indexes in case of range partitions */
-		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
-			b1->indexes[i] != b2->indexes[i])
-			return false;
+		args = lappend(args, keyCol);
 	}
-	return true;
+
+	fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+						 BOOLOID,
+						 args,
+						 InvalidOid,
+						 InvalidOid,
+						 COERCE_EXPLICIT_CALL);
+
+	return list_make1(fexpr);
 }
 
 /*
- * Return a copy of given PartitionBoundInfo structure. The data types of bounds
- * are described by given partition key specification.
+ * get_qual_for_list
+ *
+ * Returns an implicit-AND list of expressions to use as a list partition's
+ * constraint, given the partition key and bound structures.
+ *
+ * The function returns NIL for a default partition when it's the only
+ * partition since in that case there is no constraint.
  */
-extern PartitionBoundInfo
-partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key)
+static List *
+get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 {
-	PartitionBoundInfo dest;
-	int			i;
-	int			ndatums;
-	int			partnatts;
-	int			num_indexes;
+	PartitionKey key = RelationGetPartitionKey(parent);
+	List	   *result;
+	Expr	   *keyCol;
+	Expr	   *opexpr;
+	NullTest   *nulltest;
+	ListCell   *cell;
+	List	   *elems = NIL;
+	bool		list_has_null = false;
 
-	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	/*
+	 * Only single-column list partitioning is supported, so we are worried
+	 * only about the partition key with index 0.
+	 */
+	Assert(key->partnatts == 1);
 
-	dest->strategy = src->strategy;
-	ndatums = dest->ndatums = src->ndatums;
-	partnatts = key->partnatts;
+	/* Construct Var or expression representing the partition column */
+	if (key->partattrs[0] != 0)
+		keyCol = (Expr *) makeVar(1,
+								  key->partattrs[0],
+								  key->parttypid[0],
+								  key->parttypmod[0],
+								  key->parttypcoll[0],
+								  0);
+	else
+		keyCol = (Expr *) copyObject(linitial(key->partexprs));
 
-	num_indexes = get_partition_bound_num_indexes(src);
+	/*
+	 * For default list partition, collect datums for all the partitions. The
+	 * default partition constraint should check that the partition key is
+	 * equal to none of those.
+	 */
+	if (spec->is_default)
+	{
+		int			i;
+		int			ndatums = 0;
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
-	/* List partitioned tables have only a single partition key. */
-	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+		if (boundinfo)
+		{
+			ndatums = boundinfo->ndatums;
 
-	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+			if (partition_bound_accepts_nulls(boundinfo))
+				list_has_null = true;
+		}
+
+		/*
+		 * If default is the only partition, there need not be any partition
+		 * constraint on it.
+		 */
+		if (ndatums == 0 && !list_has_null)
+			return NIL;
 
-	if (src->kind != NULL)
-	{
-		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
-														 sizeof(PartitionRangeDatumKind *));
 		for (i = 0; i < ndatums; i++)
 		{
-			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
-															   sizeof(PartitionRangeDatumKind));
+			Const	   *val;
+
+			/*
+			 * Construct Const from known-not-null datum.  We must be careful
+			 * to copy the value, because our result has to be able to outlive
+			 * the relcache entry we're copying from.
+			 */
+			val = makeConst(key->parttypid[0],
+							key->parttypmod[0],
+							key->parttypcoll[0],
+							key->parttyplen[0],
+							datumCopy(*boundinfo->datums[i],
+									  key->parttypbyval[0],
+									  key->parttyplen[0]),
+							false,	/* isnull */
+							key->parttypbyval[0]);
 
-			memcpy(dest->kind[i], src->kind[i],
-				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+			elems = lappend(elems, val);
 		}
 	}
 	else
-		dest->kind = NULL;
+	{
+		/*
+		 * Create list of Consts for the allowed values, excluding any nulls.
+		 */
+		foreach(cell, spec->listdatums)
+		{
+			Const	   *val = castNode(Const, lfirst(cell));
+
+			if (val->constisnull)
+				list_has_null = true;
+			else
+				elems = lappend(elems, copyObject(val));
+		}
+	}
 
-	for (i = 0; i < ndatums; i++)
+	if (elems)
 	{
-		int			j;
+		/*
+		 * Generate the operator expression from the non-null partition
+		 * values.
+		 */
+		opexpr = make_partition_op_expr(key, 0, BTEqualStrategyNumber,
+										keyCol, (Expr *) elems);
+	}
+	else
+	{
+		/*
+		 * If there are no partition values, we don't need an operator
+		 * expression.
+		 */
+		opexpr = NULL;
+	}
 
+	if (!list_has_null)
+	{
 		/*
-		 * For a corresponding to hash partition, datums array will have two
-		 * elements - modulus and remainder.
+		 * Gin up a "col IS NOT NULL" test that will be AND'd with the main
+		 * expression.  This might seem redundant, but the partition routing
+		 * machinery needs it.
 		 */
-		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
-		int			natts = hash_part ? 2 : partnatts;
+		nulltest = makeNode(NullTest);
+		nulltest->arg = keyCol;
+		nulltest->nulltesttype = IS_NOT_NULL;
+		nulltest->argisrow = false;
+		nulltest->location = -1;
 
-		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+		result = opexpr ? list_make2(nulltest, opexpr) : list_make1(nulltest);
+	}
+	else
+	{
+		/*
+		 * Gin up a "col IS NULL" test that will be OR'd with the main
+		 * expression.
+		 */
+		nulltest = makeNode(NullTest);
+		nulltest->arg = keyCol;
+		nulltest->nulltesttype = IS_NULL;
+		nulltest->argisrow = false;
+		nulltest->location = -1;
 
-		for (j = 0; j < natts; j++)
+		if (opexpr)
 		{
-			bool		byval;
-			int			typlen;
-
-			if (hash_part)
-			{
-				typlen = sizeof(int32); /* Always int4 */
-				byval = true;	/* int4 is pass-by-value */
-			}
-			else
-			{
-				byval = key->parttypbyval[j];
-				typlen = key->parttyplen[j];
-			}
+			Expr	   *or;
 
-			if (dest->kind == NULL ||
-				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
-				dest->datums[i][j] = datumCopy(src->datums[i][j],
-											   byval, typlen);
+			or = makeBoolExpr(OR_EXPR, list_make2(nulltest, opexpr), -1);
+			result = list_make1(or);
 		}
+		else
+			result = list_make1(nulltest);
 	}
 
-	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
-	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
-
-	dest->null_index = src->null_index;
-	dest->default_index = src->default_index;
+	/*
+	 * Note that, in general, applying NOT to a constraint expression doesn't
+	 * necessarily invert the set of rows it accepts, because NOT (NULL) is
+	 * NULL.  However, the partition constraints we construct here never
+	 * evaluate to NULL, so applying NOT works as intended.
+	 */
+	if (spec->is_default)
+	{
+		result = list_make1(make_ands_explicit(result));
+		result = list_make1(makeBoolExpr(NOT_EXPR, result, -1));
+	}
 
-	return dest;
+	return result;
 }
 
 /*
- * check_new_partition_bound
+ * get_range_key_properties
+ *		Returns range partition key information for a given column
+ *
+ * This is a subroutine for get_qual_for_range, and its API is pretty
+ * specialized to that caller.
  *
- * Checks if the new partition's bound overlaps any of the existing partitions
- * of parent.  Also performs additional checks as necessary per strategy.
+ * Constructs an Expr for the key column (returned in *keyCol) and Consts
+ * for the lower and upper range limits (returned in *lower_val and
+ * *upper_val).  For MINVALUE/MAXVALUE limits, NULL is returned instead of
+ * a Const.  All of these structures are freshly palloc'd.
+ *
+ * *partexprs_item points to the cell containing the next expression in
+ * the key->partexprs list, or NULL.  It may be advanced upon return.
  */
-void
-check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec)
+static void
+get_range_key_properties(PartitionKey key, int keynum,
+						 PartitionRangeDatum *ldatum,
+						 PartitionRangeDatum *udatum,
+						 ListCell **partexprs_item,
+						 Expr **keyCol,
+						 Const **lower_val, Const **upper_val)
 {
-	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	PartitionBoundInfo boundinfo = partdesc->boundinfo;
-	ParseState *pstate = make_parsestate(NULL);
-	int			with = -1;
-	bool		overlap = false;
-
-	if (spec->is_default)
+	/* Get partition key expression for this column */
+	if (key->partattrs[keynum] != 0)
 	{
-		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
-			return;
-
-		/* Default partition already exists, error out. */
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
-				 parser_errposition(pstate, spec->location)));
+		*keyCol = (Expr *) makeVar(1,
+								   key->partattrs[keynum],
+								   key->parttypid[keynum],
+								   key->parttypmod[keynum],
+								   key->parttypcoll[keynum],
+								   0);
 	}
-
-	switch (key->strategy)
+	else
 	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					Datum	  **datums = boundinfo->datums;
-					int			ndatums = boundinfo->ndatums;
-					int			greatest_modulus;
-					int			remainder;
-					int			offset;
-					bool		valid_modulus = true;
-					int			prev_modulus,	/* Previous largest modulus */
-								next_modulus;	/* Next largest modulus */
-
-					/*
-					 * Check rule that every modulus must be a factor of the
-					 * next larger modulus.  For example, if you have a bunch
-					 * of partitions that all have modulus 5, you can add a
-					 * new partition with modulus 10 or a new partition with
-					 * modulus 15, but you cannot add both a partition with
-					 * modulus 10 and a partition with modulus 15, because 10
-					 * is not a factor of 15.
-					 *
-					 * Get the greatest (modulus, remainder) pair contained in
-					 * boundinfo->datums that is less than or equal to the
-					 * (spec->modulus, spec->remainder) pair.
-					 */
-					offset = partition_hash_bsearch(key, boundinfo,
-													spec->modulus,
-													spec->remainder);
-					if (offset < 0)
-					{
-						next_modulus = DatumGetInt32(datums[0][0]);
-						valid_modulus = (next_modulus % spec->modulus) == 0;
-					}
-					else
-					{
-						prev_modulus = DatumGetInt32(datums[offset][0]);
-						valid_modulus = (spec->modulus % prev_modulus) == 0;
-
-						if (valid_modulus && (offset + 1) < ndatums)
-						{
-							next_modulus = DatumGetInt32(datums[offset + 1][0]);
-							valid_modulus = (next_modulus % spec->modulus) == 0;
-						}
-					}
-
-					if (!valid_modulus)
-						ereport(ERROR,
-								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
-
-					greatest_modulus = get_greatest_modulus(boundinfo);
-					remainder = spec->remainder;
-
-					/*
-					 * Normally, the lowest remainder that could conflict with
-					 * the new partition is equal to the remainder specified
-					 * for the new partition, but when the new partition has a
-					 * modulus higher than any used so far, we need to adjust.
-					 */
-					if (remainder >= greatest_modulus)
-						remainder = remainder % greatest_modulus;
-
-					/* Check every potentially-conflicting remainder. */
-					do
-					{
-						if (boundinfo->indexes[remainder] != -1)
-						{
-							overlap = true;
-							with = boundinfo->indexes[remainder];
-							break;
-						}
-						remainder += spec->modulus;
-					} while (remainder < greatest_modulus);
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_LIST:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+		if (*partexprs_item == NULL)
+			elog(ERROR, "wrong number of partition key expressions");
+		*keyCol = copyObject(lfirst(*partexprs_item));
+		*partexprs_item = lnext(*partexprs_item);
+	}
 
-				if (partdesc->nparts > 0)
-				{
-					ListCell   *cell;
+	/* Get appropriate Const nodes for the bounds */
+	if (ldatum->kind == PARTITION_RANGE_DATUM_VALUE)
+		*lower_val = castNode(Const, copyObject(ldatum->value));
+	else
+		*lower_val = NULL;
 
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo) ||
-							partition_bound_has_default(boundinfo)));
+	if (udatum->kind == PARTITION_RANGE_DATUM_VALUE)
+		*upper_val = castNode(Const, copyObject(udatum->value));
+	else
+		*upper_val = NULL;
+}
 
-					foreach(cell, spec->listdatums)
-					{
-						Const	   *val = castNode(Const, lfirst(cell));
-
-						if (!val->constisnull)
-						{
-							int			offset;
-							bool		equal;
-
-							offset = partition_list_bsearch(key, boundinfo,
-															val->constvalue,
-															&equal);
-							if (offset >= 0 && equal)
-							{
-								overlap = true;
-								with = boundinfo->indexes[offset];
-								break;
-							}
-						}
-						else if (partition_bound_accepts_nulls(boundinfo))
-						{
-							overlap = true;
-							with = boundinfo->null_index;
-							break;
-						}
-					}
-				}
+ /*
+  * get_range_nulltest
+  *
+  * A non-default range partition table does not currently allow partition
+  * keys to be null, so emit an IS NOT NULL expression for each key column.
+  */
+static List *
+get_range_nulltest(PartitionKey key)
+{
+	List	   *result = NIL;
+	NullTest   *nulltest;
+	ListCell   *partexprs_item;
+	int			i;
 
-				break;
-			}
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		Expr	   *keyCol;
 
-		case PARTITION_STRATEGY_RANGE:
-			{
-				PartitionRangeBound *lower,
-						   *upper;
-
-				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
-				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
-
-				/*
-				 * First check if the resulting range would be empty with
-				 * specified lower and upper bounds
-				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
-				{
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-							 errmsg("empty range bound specified for partition \"%s\"",
-									relname),
-							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
-									   get_range_partbound_string(spec->lowerdatums),
-									   get_range_partbound_string(spec->upperdatums)),
-							 parser_errposition(pstate, spec->location)));
-				}
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					int			offset;
-					bool		equal;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_has_default(boundinfo)));
-
-					/*
-					 * Test whether the new lower bound (which is treated
-					 * inclusively as part of the new partition) lies inside
-					 * an existing partition, or in a gap.
-					 *
-					 * If it's inside an existing partition, the bound at
-					 * offset + 1 will be the upper bound of that partition,
-					 * and its index will be >= 0.
-					 *
-					 * If it's in a gap, the bound at offset + 1 will be the
-					 * lower bound of the next partition, and its index will
-					 * be -1. This is also true if there is no next partition,
-					 * since the index array is initialised with an extra -1
-					 * at the end.
-					 */
-					offset = partition_range_bsearch(key, boundinfo, lower,
-													 &equal);
-
-					if (boundinfo->indexes[offset + 1] < 0)
-					{
-						/*
-						 * Check that the new partition will fit in the gap.
-						 * For it to fit, the new upper bound must be less
-						 * than or equal to the lower bound of the next
-						 * partition, if there is one.
-						 */
-						if (offset + 1 < boundinfo->ndatums)
-						{
-							int32		cmpval;
-							Datum 	   *datums;
-							PartitionRangeDatumKind *kind;
-							bool		is_lower;
-
-							datums = boundinfo->datums[offset + 1];
-							kind = boundinfo->kind[offset + 1];
-							is_lower = (boundinfo->indexes[offset + 1] == -1);
-
-							cmpval = partition_rbound_cmp(key, datums, kind,
-														  is_lower, upper);
-							if (cmpval < 0)
-							{
-								/*
-								 * The new partition overlaps with the
-								 * existing partition between offset + 1 and
-								 * offset + 2.
-								 */
-								overlap = true;
-								with = boundinfo->indexes[offset + 2];
-							}
-						}
-					}
-					else
-					{
-						/*
-						 * The new partition overlaps with the existing
-						 * partition between offset and offset + 1.
-						 */
-						overlap = true;
-						with = boundinfo->indexes[offset + 1];
-					}
-				}
-
-				break;
-			}
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	if (overlap)
-	{
-		Assert(with >= 0);
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" would overlap partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[with])),
-				 parser_errposition(pstate, spec->location)));
-	}
-}
-
-/*
- * check_default_allows_bound
- *
- * This function checks if there exists a row in the default partition that
- * would properly belong to the new partition being added.  If it finds one,
- * it throws an error.
- */
-void
-check_default_allows_bound(Relation parent, Relation default_rel,
-						   PartitionBoundSpec *new_spec)
-{
-	List	   *new_part_constraints;
-	List	   *def_part_constraints;
-	List	   *all_parts;
-	ListCell   *lc;
-
-	new_part_constraints = (new_spec->strategy == PARTITION_STRATEGY_LIST)
-		? get_qual_for_list(parent, new_spec)
-		: get_qual_for_range(parent, new_spec, false);
-	def_part_constraints =
-		get_proposed_default_constraint(new_part_constraints);
-
-	/*
-	 * If the existing constraints on the default partition imply that it will
-	 * not contain any row that would belong to the new partition, we can
-	 * avoid scanning the default partition.
-	 */
-	if (PartConstraintImpliedByRelConstraint(default_rel, def_part_constraints))
-	{
-		ereport(INFO,
-				(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
-						RelationGetRelationName(default_rel))));
-		return;
-	}
-
-	/*
-	 * Scan the default partition and its subpartitions, and check for rows
-	 * that do not satisfy the revised partition constraints.
-	 */
-	if (default_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-		all_parts = find_all_inheritors(RelationGetRelid(default_rel),
-										AccessExclusiveLock, NULL);
-	else
-		all_parts = list_make1_oid(RelationGetRelid(default_rel));
-
-	foreach(lc, all_parts)
-	{
-		Oid			part_relid = lfirst_oid(lc);
-		Relation	part_rel;
-		Expr	   *constr;
-		Expr	   *partition_constraint;
-		EState	   *estate;
-		HeapTuple	tuple;
-		ExprState  *partqualstate = NULL;
-		Snapshot	snapshot;
-		TupleDesc	tupdesc;
-		ExprContext *econtext;
-		HeapScanDesc scan;
-		MemoryContext oldCxt;
-		TupleTableSlot *tupslot;
-
-		/* Lock already taken above. */
-		if (part_relid != RelationGetRelid(default_rel))
-		{
-			part_rel = heap_open(part_relid, NoLock);
-
-			/*
-			 * If the partition constraints on default partition child imply
-			 * that it will not contain any row that would belong to the new
-			 * partition, we can avoid scanning the child table.
-			 */
-			if (PartConstraintImpliedByRelConstraint(part_rel,
-													 def_part_constraints))
-			{
-				ereport(INFO,
-						(errmsg("updated partition constraint for default partition \"%s\" is implied by existing constraints",
-								RelationGetRelationName(part_rel))));
-
-				heap_close(part_rel, NoLock);
-				continue;
-			}
-		}
-		else
-			part_rel = default_rel;
-
-		/*
-		 * Only RELKIND_RELATION relations (i.e. leaf partitions) need to be
-		 * scanned.
-		 */
-		if (part_rel->rd_rel->relkind != RELKIND_RELATION)
-		{
-			if (part_rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
-				ereport(WARNING,
-						(errcode(ERRCODE_CHECK_VIOLATION),
-						 errmsg("skipped scanning foreign table \"%s\" which is a partition of default partition \"%s\"",
-								RelationGetRelationName(part_rel),
-								RelationGetRelationName(default_rel))));
-
-			if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
-				heap_close(part_rel, NoLock);
-
-			continue;
-		}
-
-		tupdesc = CreateTupleDescCopy(RelationGetDescr(part_rel));
-		constr = linitial(def_part_constraints);
-		partition_constraint = (Expr *)
-			map_partition_varattnos((List *) constr,
-									1, part_rel, parent, NULL);
-		estate = CreateExecutorState();
-
-		/* Build expression execution states for partition check quals */
-		partqualstate = ExecPrepareExpr(partition_constraint, estate);
-
-		econtext = GetPerTupleExprContext(estate);
-		snapshot = RegisterSnapshot(GetLatestSnapshot());
-		scan = heap_beginscan(part_rel, snapshot, 0, NULL);
-		tupslot = MakeSingleTupleTableSlot(tupdesc);
-
-		/*
-		 * Switch to per-tuple memory context and reset it for each tuple
-		 * produced, so we don't leak memory.
-		 */
-		oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
-
-		while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
-		{
-			ExecStoreTuple(tuple, tupslot, InvalidBuffer, false);
-			econtext->ecxt_scantuple = tupslot;
-
-			if (!ExecCheck(partqualstate, econtext))
-				ereport(ERROR,
-						(errcode(ERRCODE_CHECK_VIOLATION),
-						 errmsg("updated partition constraint for default partition \"%s\" would be violated by some row",
-								RelationGetRelationName(default_rel))));
-
-			ResetExprContext(econtext);
-			CHECK_FOR_INTERRUPTS();
-		}
-
-		MemoryContextSwitchTo(oldCxt);
-		heap_endscan(scan);
-		UnregisterSnapshot(snapshot);
-		ExecDropSingleTupleTableSlot(tupslot);
-		FreeExecutorState(estate);
-
-		if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
-			heap_close(part_rel, NoLock);	/* keep the lock until commit */
-	}
-}
-
-/*
- * get_partition_parent
- *
- * Returns inheritance parent of a partition by scanning pg_inherits
- *
- * Note: Because this function assumes that the relation whose OID is passed
- * as an argument will have precisely one parent, it should only be called
- * when it is known that the relation is a partition.
- */
-Oid
-get_partition_parent(Oid relid)
-{
-	Form_pg_inherits form;
-	Relation	catalogRelation;
-	SysScanDesc scan;
-	ScanKeyData key[2];
-	HeapTuple	tuple;
-	Oid			result;
-
-	catalogRelation = heap_open(InheritsRelationId, AccessShareLock);
-
-	ScanKeyInit(&key[0],
-				Anum_pg_inherits_inhrelid,
-				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(relid));
-	ScanKeyInit(&key[1],
-				Anum_pg_inherits_inhseqno,
-				BTEqualStrategyNumber, F_INT4EQ,
-				Int32GetDatum(1));
-
-	scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId, true,
-							  NULL, 2, key);
-
-	tuple = systable_getnext(scan);
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "could not find tuple for parent of relation %u", relid);
-
-	form = (Form_pg_inherits) GETSTRUCT(tuple);
-	result = form->inhparent;
-
-	systable_endscan(scan);
-	heap_close(catalogRelation, AccessShareLock);
-
-	return result;
-}
-
-/*
- * get_qual_from_partbound
- *		Given a parser node for partition bound, return the list of executable
- *		expressions as partition constraint
- */
-List *
-get_qual_from_partbound(Relation rel, Relation parent,
-						PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	List	   *my_qual = NIL;
-
-	Assert(key != NULL);
-
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-			my_qual = get_qual_for_hash(parent, spec);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-			my_qual = get_qual_for_list(parent, spec);
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-			my_qual = get_qual_for_range(parent, spec, false);
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	return my_qual;
-}
-
-/*
- * map_partition_varattnos - maps varattno of any Vars in expr from the
- * attno's of 'from_rel' to the attno's of 'to_rel' partition, each of which
- * may be either a leaf partition or a partitioned table, but both of which
- * must be from the same partitioning hierarchy.
- *
- * Even though all of the same column names must be present in all relations
- * in the hierarchy, and they must also have the same types, the attnos may
- * be different.
- *
- * If found_whole_row is not NULL, *found_whole_row returns whether a
- * whole-row variable was found in the input expression.
- *
- * Note: this will work on any node tree, so really the argument and result
- * should be declared "Node *".  But a substantial majority of the callers
- * are working on Lists, so it's less messy to do the casts internally.
- */
-List *
-map_partition_varattnos(List *expr, int fromrel_varno,
-						Relation to_rel, Relation from_rel,
-						bool *found_whole_row)
-{
-	bool		my_found_whole_row = false;
-
-	if (expr != NIL)
-	{
-		AttrNumber *part_attnos;
-
-		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,
-											fromrel_varno, 0,
-											part_attnos,
-											RelationGetDescr(from_rel)->natts,
-											RelationGetForm(to_rel)->reltype,
-											&my_found_whole_row);
-	}
-
-	if (found_whole_row)
-		*found_whole_row = my_found_whole_row;
-
-	return expr;
-}
-
-/*
- * RelationGetPartitionQual
- *
- * Returns a list of partition quals
- */
-List *
-RelationGetPartitionQual(Relation rel)
-{
-	/* Quick exit */
-	if (!rel->rd_rel->relispartition)
-		return NIL;
-
-	return generate_partition_qual(rel);
-}
-
-/*
- * get_partition_qual_relid
- *
- * Returns an expression tree describing the passed-in relation's partition
- * constraint. If there is no partition constraint returns NULL; this can
- * happen if the default partition is the only partition.
- */
-Expr *
-get_partition_qual_relid(Oid relid)
-{
-	Relation	rel = heap_open(relid, AccessShareLock);
-	Expr	   *result = NULL;
-	List	   *and_args;
-
-	/* Do the work only if this relation is a partition. */
-	if (rel->rd_rel->relispartition)
-	{
-		and_args = generate_partition_qual(rel);
-
-		if (and_args == NIL)
-			result = NULL;
-		else if (list_length(and_args) > 1)
-			result = makeBoolExpr(AND_EXPR, and_args, -1);
-		else
-			result = linitial(and_args);
-	}
-
-	/* Keep the lock. */
-	heap_close(rel, NoLock);
-
-	return result;
-}
-
-/* Module-local functions */
-
-/*
- * get_partition_operator
- *
- * Return oid of the operator of given strategy for a given partition key
- * column.
- */
-static Oid
-get_partition_operator(PartitionKey key, int col, StrategyNumber strategy,
-					   bool *need_relabel)
-{
-	Oid			operoid;
-
-	/*
-	 * First check if there exists an operator of the given strategy, with
-	 * this column's type as both its lefttype and righttype, in the
-	 * partitioning operator family specified for the column.
-	 */
-	operoid = get_opfamily_member(key->partopfamily[col],
-								  key->parttypid[col],
-								  key->parttypid[col],
-								  strategy);
-
-	/*
-	 * If one doesn't exist, we must resort to using an operator in the same
-	 * operator family but with the operator class declared input type.  It is
-	 * OK to do so, because the column's type is known to be binary-coercible
-	 * with the operator class input type (otherwise, the operator class in
-	 * question would not have been accepted as the partitioning operator
-	 * class).  We must however inform the caller to wrap the non-Const
-	 * expression with a RelabelType node to denote the implicit coercion. It
-	 * ensures that the resulting expression structurally matches similarly
-	 * processed expressions within the optimizer.
-	 */
-	if (!OidIsValid(operoid))
-	{
-		operoid = get_opfamily_member(key->partopfamily[col],
-									  key->partopcintype[col],
-									  key->partopcintype[col],
-									  strategy);
-		if (!OidIsValid(operoid))
-			elog(ERROR, "missing operator %d(%u,%u) in opfamily %u",
-				 strategy, key->partopcintype[col], key->partopcintype[col],
-				 key->partopfamily[col]);
-		*need_relabel = true;
-	}
-	else
-		*need_relabel = false;
-
-	return operoid;
-}
-
-/*
- * make_partition_op_expr
- *		Returns an Expr for the given partition key column with arg1 and
- *		arg2 as its leftop and rightop, respectively
- */
-static Expr *
-make_partition_op_expr(PartitionKey key, int keynum,
-					   uint16 strategy, Expr *arg1, Expr *arg2)
-{
-	Oid			operoid;
-	bool		need_relabel = false;
-	Expr	   *result = NULL;
-
-	/* Get the correct btree operator for this partitioning column */
-	operoid = get_partition_operator(key, keynum, strategy, &need_relabel);
-
-	/*
-	 * Chosen operator may be such that the non-Const operand needs to be
-	 * coerced, so apply the same; see the comment in
-	 * get_partition_operator().
-	 */
-	if (!IsA(arg1, Const) &&
-		(need_relabel ||
-		 key->partcollation[keynum] != key->parttypcoll[keynum]))
-		arg1 = (Expr *) makeRelabelType(arg1,
-										key->partopcintype[keynum],
-										-1,
-										key->partcollation[keynum],
-										COERCE_EXPLICIT_CAST);
-
-	/* Generate the actual expression */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_LIST:
-			{
-				List	   *elems = (List *) arg2;
-				int			nelems = list_length(elems);
-
-				Assert(nelems >= 1);
-				Assert(keynum == 0);
-
-				if (nelems > 1 &&
-					!type_is_array(key->parttypid[keynum]))
-				{
-					ArrayExpr  *arrexpr;
-					ScalarArrayOpExpr *saopexpr;
-
-					/* Construct an ArrayExpr for the right-hand inputs */
-					arrexpr = makeNode(ArrayExpr);
-					arrexpr->array_typeid =
-									get_array_type(key->parttypid[keynum]);
-					arrexpr->array_collid = key->parttypcoll[keynum];
-					arrexpr->element_typeid = key->parttypid[keynum];
-					arrexpr->elements = elems;
-					arrexpr->multidims = false;
-					arrexpr->location = -1;
-
-					/* Build leftop = ANY (rightop) */
-					saopexpr = makeNode(ScalarArrayOpExpr);
-					saopexpr->opno = operoid;
-					saopexpr->opfuncid = get_opcode(operoid);
-					saopexpr->useOr = true;
-					saopexpr->inputcollid = key->partcollation[keynum];
-					saopexpr->args = list_make2(arg1, arrexpr);
-					saopexpr->location = -1;
-
-					result = (Expr *) saopexpr;
-				}
-				else
-				{
-					List	   *elemops = NIL;
-					ListCell   *lc;
-
-					foreach (lc, elems)
-					{
-						Expr   *elem = lfirst(lc),
-							   *elemop;
-
-						elemop = make_opclause(operoid,
-											   BOOLOID,
-											   false,
-											   arg1, elem,
-											   InvalidOid,
-											   key->partcollation[keynum]);
-						elemops = lappend(elemops, elemop);
-					}
-
-					result = nelems > 1 ? makeBoolExpr(OR_EXPR, elemops, -1) : linitial(elemops);
-				}
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			result = make_opclause(operoid,
-								   BOOLOID,
-								   false,
-								   arg1, arg2,
-								   InvalidOid,
-								   key->partcollation[keynum]);
-			break;
-
-		default:
-			elog(ERROR, "invalid partitioning strategy");
-			break;
-	}
-
-	return result;
-}
-
-/*
- * get_qual_for_hash
- *
- * Given a list of partition columns, modulus and remainder corresponding to a
- * partition, this function returns CHECK constraint expression Node for that
- * partition.
- *
- * The partition constraint for a hash partition is always a call to the
- * built-in function satisfies_hash_partition().  The first two arguments are
- * the modulus and remainder for the partition; the remaining arguments are the
- * values to be hashed.
- */
-static List *
-get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	FuncExpr   *fexpr;
-	Node	   *relidConst;
-	Node	   *modulusConst;
-	Node	   *remainderConst;
-	List	   *args;
-	ListCell   *partexprs_item;
-	int			i;
-
-	/* Fixed arguments. */
-	relidConst = (Node *) makeConst(OIDOID,
-									-1,
-									InvalidOid,
-									sizeof(Oid),
-									ObjectIdGetDatum(RelationGetRelid(parent)),
-									false,
-									true);
-
-	modulusConst = (Node *) makeConst(INT4OID,
-									  -1,
-									  InvalidOid,
-									  sizeof(int32),
-									  Int32GetDatum(spec->modulus),
-									  false,
-									  true);
-
-	remainderConst = (Node *) makeConst(INT4OID,
-										-1,
-										InvalidOid,
-										sizeof(int32),
-										Int32GetDatum(spec->remainder),
-										false,
-										true);
-
-	args = list_make3(relidConst, modulusConst, remainderConst);
-	partexprs_item = list_head(key->partexprs);
-
-	/* Add an argument for each key column. */
-	for (i = 0; i < key->partnatts; i++)
-	{
-		Node	   *keyCol;
-
-		/* Left operand */
-		if (key->partattrs[i] != 0)
-		{
-			keyCol = (Node *) makeVar(1,
-									  key->partattrs[i],
-									  key->parttypid[i],
-									  key->parttypmod[i],
-									  key->parttypcoll[i],
-									  0);
-		}
-		else
-		{
-			keyCol = (Node *) copyObject(lfirst(partexprs_item));
-			partexprs_item = lnext(partexprs_item);
-		}
-
-		args = lappend(args, keyCol);
-	}
-
-	fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
-						 BOOLOID,
-						 args,
-						 InvalidOid,
-						 InvalidOid,
-						 COERCE_EXPLICIT_CALL);
-
-	return list_make1(fexpr);
-}
-
-/*
- * get_qual_for_list
- *
- * Returns an implicit-AND list of expressions to use as a list partition's
- * constraint, given the partition key and bound structures.
- *
- * The function returns NIL for a default partition when it's the only
- * partition since in that case there is no constraint.
- */
-static List *
-get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	List	   *result;
-	Expr	   *keyCol;
-	Expr	   *opexpr;
-	NullTest   *nulltest;
-	ListCell   *cell;
-	List	   *elems = NIL;
-	bool		list_has_null = false;
-
-	/*
-	 * Only single-column list partitioning is supported, so we are worried
-	 * only about the partition key with index 0.
-	 */
-	Assert(key->partnatts == 1);
-
-	/* Construct Var or expression representing the partition column */
-	if (key->partattrs[0] != 0)
-		keyCol = (Expr *) makeVar(1,
-								  key->partattrs[0],
-								  key->parttypid[0],
-								  key->parttypmod[0],
-								  key->parttypcoll[0],
-								  0);
-	else
-		keyCol = (Expr *) copyObject(linitial(key->partexprs));
-
-	/*
-	 * For default list partition, collect datums for all the partitions. The
-	 * default partition constraint should check that the partition key is
-	 * equal to none of those.
-	 */
-	if (spec->is_default)
-	{
-		int			i;
-		int			ndatums = 0;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
-		PartitionBoundInfo boundinfo = pdesc->boundinfo;
-
-		if (boundinfo)
-		{
-			ndatums = boundinfo->ndatums;
-
-			if (partition_bound_accepts_nulls(boundinfo))
-				list_has_null = true;
-		}
-
-		/*
-		 * If default is the only partition, there need not be any partition
-		 * constraint on it.
-		 */
-		if (ndatums == 0 && !list_has_null)
-			return NIL;
-
-		for (i = 0; i < ndatums; i++)
-		{
-			Const	   *val;
-
-			/*
-			 * Construct Const from known-not-null datum.  We must be careful
-			 * to copy the value, because our result has to be able to outlive
-			 * the relcache entry we're copying from.
-			 */
-			val = makeConst(key->parttypid[0],
-							key->parttypmod[0],
-							key->parttypcoll[0],
-							key->parttyplen[0],
-							datumCopy(*boundinfo->datums[i],
-									  key->parttypbyval[0],
-									  key->parttyplen[0]),
-							false,	/* isnull */
-							key->parttypbyval[0]);
-
-			elems = lappend(elems, val);
-		}
-	}
-	else
-	{
-		/*
-		 * Create list of Consts for the allowed values, excluding any nulls.
-		 */
-		foreach(cell, spec->listdatums)
-		{
-			Const	   *val = castNode(Const, lfirst(cell));
-
-			if (val->constisnull)
-				list_has_null = true;
-			else
-				elems = lappend(elems, copyObject(val));
-		}
-	}
-
-	if (elems)
-	{
-		/*
-		 * Generate the operator expression from the non-null partition
-		 * values.
-		 */
-		opexpr = make_partition_op_expr(key, 0, BTEqualStrategyNumber,
-										keyCol, (Expr *) elems);
-	}
-	else
-	{
-		/*
-		 * If there are no partition values, we don't need an operator
-		 * expression.
-		 */
-		opexpr = NULL;
-	}
-
-	if (!list_has_null)
-	{
-		/*
-		 * Gin up a "col IS NOT NULL" test that will be AND'd with the main
-		 * expression.  This might seem redundant, but the partition routing
-		 * machinery needs it.
-		 */
-		nulltest = makeNode(NullTest);
-		nulltest->arg = keyCol;
-		nulltest->nulltesttype = IS_NOT_NULL;
-		nulltest->argisrow = false;
-		nulltest->location = -1;
-
-		result = opexpr ? list_make2(nulltest, opexpr) : list_make1(nulltest);
-	}
-	else
-	{
-		/*
-		 * Gin up a "col IS NULL" test that will be OR'd with the main
-		 * expression.
-		 */
-		nulltest = makeNode(NullTest);
-		nulltest->arg = keyCol;
-		nulltest->nulltesttype = IS_NULL;
-		nulltest->argisrow = false;
-		nulltest->location = -1;
-
-		if (opexpr)
-		{
-			Expr	   *or;
-
-			or = makeBoolExpr(OR_EXPR, list_make2(nulltest, opexpr), -1);
-			result = list_make1(or);
-		}
-		else
-			result = list_make1(nulltest);
-	}
-
-	/*
-	 * Note that, in general, applying NOT to a constraint expression doesn't
-	 * necessarily invert the set of rows it accepts, because NOT (NULL) is
-	 * NULL.  However, the partition constraints we construct here never
-	 * evaluate to NULL, so applying NOT works as intended.
-	 */
-	if (spec->is_default)
-	{
-		result = list_make1(make_ands_explicit(result));
-		result = list_make1(makeBoolExpr(NOT_EXPR, result, -1));
-	}
-
-	return result;
-}
-
-/*
- * get_range_key_properties
- *		Returns range partition key information for a given column
- *
- * This is a subroutine for get_qual_for_range, and its API is pretty
- * specialized to that caller.
- *
- * Constructs an Expr for the key column (returned in *keyCol) and Consts
- * for the lower and upper range limits (returned in *lower_val and
- * *upper_val).  For MINVALUE/MAXVALUE limits, NULL is returned instead of
- * a Const.  All of these structures are freshly palloc'd.
- *
- * *partexprs_item points to the cell containing the next expression in
- * the key->partexprs list, or NULL.  It may be advanced upon return.
- */
-static void
-get_range_key_properties(PartitionKey key, int keynum,
-						 PartitionRangeDatum *ldatum,
-						 PartitionRangeDatum *udatum,
-						 ListCell **partexprs_item,
-						 Expr **keyCol,
-						 Const **lower_val, Const **upper_val)
-{
-	/* Get partition key expression for this column */
-	if (key->partattrs[keynum] != 0)
-	{
-		*keyCol = (Expr *) makeVar(1,
-								   key->partattrs[keynum],
-								   key->parttypid[keynum],
-								   key->parttypmod[keynum],
-								   key->parttypcoll[keynum],
-								   0);
-	}
-	else
-	{
-		if (*partexprs_item == NULL)
-			elog(ERROR, "wrong number of partition key expressions");
-		*keyCol = copyObject(lfirst(*partexprs_item));
-		*partexprs_item = lnext(*partexprs_item);
-	}
-
-	/* Get appropriate Const nodes for the bounds */
-	if (ldatum->kind == PARTITION_RANGE_DATUM_VALUE)
-		*lower_val = castNode(Const, copyObject(ldatum->value));
-	else
-		*lower_val = NULL;
-
-	if (udatum->kind == PARTITION_RANGE_DATUM_VALUE)
-		*upper_val = castNode(Const, copyObject(udatum->value));
-	else
-		*upper_val = NULL;
-}
-
- /*
-  * get_range_nulltest
-  *
-  * A non-default range partition table does not currently allow partition
-  * keys to be null, so emit an IS NOT NULL expression for each key column.
-  */
-static List *
-get_range_nulltest(PartitionKey key)
-{
-	List	   *result = NIL;
-	NullTest   *nulltest;
-	ListCell   *partexprs_item;
-	int			i;
-
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		Expr	   *keyCol;
-
-		if (key->partattrs[i] != 0)
-		{
-			keyCol = (Expr *) makeVar(1,
-									  key->partattrs[i],
-									  key->parttypid[i],
-									  key->parttypmod[i],
-									  key->parttypcoll[i],
-									  0);
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-			keyCol = copyObject(lfirst(partexprs_item));
-			partexprs_item = lnext(partexprs_item);
-		}
-
-		nulltest = makeNode(NullTest);
-		nulltest->arg = keyCol;
-		nulltest->nulltesttype = IS_NOT_NULL;
-		nulltest->argisrow = false;
-		nulltest->location = -1;
-		result = lappend(result, nulltest);
-	}
-
-	return result;
-}
-
-/*
- * get_qual_for_range
- *
- * Returns an implicit-AND list of expressions to use as a range partition's
- * constraint, given the partition key and bound structures.
- *
- * For a multi-column range partition key, say (a, b, c), with (al, bl, cl)
- * as the lower bound tuple and (au, bu, cu) as the upper bound tuple, we
- * generate an expression tree of the following form:
- *
- *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
- *		AND
- *	(a > al OR (a = al AND b > bl) OR (a = al AND b = bl AND c >= cl))
- *		AND
- *	(a < au OR (a = au AND b < bu) OR (a = au AND b = bu AND c < cu))
- *
- * It is often the case that a prefix of lower and upper bound tuples contains
- * the same values, for example, (al = au), in which case, we will emit an
- * expression tree of the following form:
- *
- *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
- *		AND
- *	(a = al)
- *		AND
- *	(b > bl OR (b = bl AND c >= cl))
- *		AND
- *	(b < bu) OR (b = bu AND c < cu))
- *
- * If a bound datum is either MINVALUE or MAXVALUE, these expressions are
- * simplified using the fact that any value is greater than MINVALUE and less
- * than MAXVALUE. So, for example, if cu = MAXVALUE, c < cu is automatically
- * true, and we need not emit any expression for it, and the last line becomes
- *
- *	(b < bu) OR (b = bu), which is simplified to (b <= bu)
- *
- * In most common cases with only one partition column, say a, the following
- * expression tree will be generated: a IS NOT NULL AND a >= al AND a < au
- *
- * For default partition, it returns the negation of the constraints of all
- * the other partitions.
- *
- * External callers should pass for_default as false; we set it to true only
- * when recursing.
- */
-static List *
-get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
-				   bool for_default)
-{
-	List	   *result = NIL;
-	ListCell   *cell1,
-			   *cell2,
-			   *partexprs_item,
-			   *partexprs_item_saved;
-	int			i,
-				j;
-	PartitionRangeDatum *ldatum,
-			   *udatum;
-	PartitionKey key = RelationGetPartitionKey(parent);
-	Expr	   *keyCol;
-	Const	   *lower_val,
-			   *upper_val;
-	List	   *lower_or_arms,
-			   *upper_or_arms;
-	int			num_or_arms,
-				current_or_arm;
-	ListCell   *lower_or_start_datum,
-			   *upper_or_start_datum;
-	bool		need_next_lower_arm,
-				need_next_upper_arm;
-
-	if (spec->is_default)
-	{
-		List	   *or_expr_args = NIL;
-		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
-		Oid		   *inhoids = pdesc->oids;
-		int			nparts = pdesc->nparts,
-					i;
-
-		for (i = 0; i < nparts; i++)
-		{
-			Oid			inhrelid = inhoids[i];
-			HeapTuple	tuple;
-			Datum		datum;
-			bool		isnull;
-			PartitionBoundSpec *bspec;
-
-			tuple = SearchSysCache1(RELOID, inhrelid);
-			if (!HeapTupleIsValid(tuple))
-				elog(ERROR, "cache lookup failed for relation %u", inhrelid);
-
-			datum = SysCacheGetAttr(RELOID, tuple,
-									Anum_pg_class_relpartbound,
-									&isnull);
-
-			Assert(!isnull);
-			bspec = (PartitionBoundSpec *)
-				stringToNode(TextDatumGetCString(datum));
-			if (!IsA(bspec, PartitionBoundSpec))
-				elog(ERROR, "expected PartitionBoundSpec");
-
-			if (!bspec->is_default)
-			{
-				List	   *part_qual;
-
-				part_qual = get_qual_for_range(parent, bspec, true);
-
-				/*
-				 * AND the constraints of the partition and add to
-				 * or_expr_args
-				 */
-				or_expr_args = lappend(or_expr_args, list_length(part_qual) > 1
-									   ? makeBoolExpr(AND_EXPR, part_qual, -1)
-									   : linitial(part_qual));
-			}
-			ReleaseSysCache(tuple);
-		}
-
-		if (or_expr_args != NIL)
-		{
-			Expr	   *other_parts_constr;
-
-			/*
-			 * Combine the constraints obtained for non-default partitions
-			 * using OR.  As requested, each of the OR's args doesn't include
-			 * the NOT NULL test for partition keys (which is to avoid its
-			 * useless repetition).  Add the same now.
-			 */
-			other_parts_constr =
-				makeBoolExpr(AND_EXPR,
-							 lappend(get_range_nulltest(key),
-									 list_length(or_expr_args) > 1
-									 ? makeBoolExpr(OR_EXPR, or_expr_args,
-													-1)
-									 : linitial(or_expr_args)),
-							 -1);
-
-			/*
-			 * Finally, the default partition contains everything *NOT*
-			 * contained in the non-default partitions.
-			 */
-			result = list_make1(makeBoolExpr(NOT_EXPR,
-											 list_make1(other_parts_constr), -1));
-		}
-
-		return result;
-	}
-
-	lower_or_start_datum = list_head(spec->lowerdatums);
-	upper_or_start_datum = list_head(spec->upperdatums);
-	num_or_arms = key->partnatts;
-
-	/*
-	 * If it is the recursive call for default, we skip the get_range_nulltest
-	 * to avoid accumulating the NullTest on the same keys for each partition.
-	 */
-	if (!for_default)
-		result = get_range_nulltest(key);
-
-	/*
-	 * Iterate over the key columns and check if the corresponding lower and
-	 * upper datums are equal using the btree equality operator for the
-	 * column's type.  If equal, we emit single keyCol = common_value
-	 * expression.  Starting from the first column for which the corresponding
-	 * lower and upper bound datums are not equal, we generate OR expressions
-	 * as shown in the function's header comment.
-	 */
-	i = 0;
-	partexprs_item = list_head(key->partexprs);
-	partexprs_item_saved = partexprs_item;	/* placate compiler */
-	forboth(cell1, spec->lowerdatums, cell2, spec->upperdatums)
-	{
-		EState	   *estate;
-		MemoryContext oldcxt;
-		Expr	   *test_expr;
-		ExprState  *test_exprstate;
-		Datum		test_result;
-		bool		isNull;
-
-		ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
-		udatum = castNode(PartitionRangeDatum, lfirst(cell2));
-
-		/*
-		 * Since get_range_key_properties() modifies partexprs_item, and we
-		 * might need to start over from the previous expression in the later
-		 * part of this function, save away the current value.
-		 */
-		partexprs_item_saved = partexprs_item;
-
-		get_range_key_properties(key, i, ldatum, udatum,
-								 &partexprs_item,
-								 &keyCol,
-								 &lower_val, &upper_val);
-
-		/*
-		 * If either value is NULL, the corresponding partition bound is
-		 * either MINVALUE or MAXVALUE, and we treat them as unequal, because
-		 * even if they're the same, there is no common value to equate the
-		 * key column with.
-		 */
-		if (!lower_val || !upper_val)
-			break;
-
-		/* Create the test expression */
-		estate = CreateExecutorState();
-		oldcxt = MemoryContextSwitchTo(estate->es_query_cxt);
-		test_expr = make_partition_op_expr(key, i, BTEqualStrategyNumber,
-										   (Expr *) lower_val,
-										   (Expr *) upper_val);
-		fix_opfuncids((Node *) test_expr);
-		test_exprstate = ExecInitExpr(test_expr, NULL);
-		test_result = ExecEvalExprSwitchContext(test_exprstate,
-												GetPerTupleExprContext(estate),
-												&isNull);
-		MemoryContextSwitchTo(oldcxt);
-		FreeExecutorState(estate);
-
-		/* If not equal, go generate the OR expressions */
-		if (!DatumGetBool(test_result))
-			break;
-
-		/*
-		 * The bounds for the last key column can't be equal, because such a
-		 * range partition would never be allowed to be defined (it would have
-		 * an empty range otherwise).
-		 */
-		if (i == key->partnatts - 1)
-			elog(ERROR, "invalid range bound specification");
-
-		/* Equal, so generate keyCol = lower_val expression */
-		result = lappend(result,
-						 make_partition_op_expr(key, i, BTEqualStrategyNumber,
-												keyCol, (Expr *) lower_val));
-
-		i++;
-	}
-
-	/* First pair of lower_val and upper_val that are not equal. */
-	lower_or_start_datum = cell1;
-	upper_or_start_datum = cell2;
-
-	/* OR will have as many arms as there are key columns left. */
-	num_or_arms = key->partnatts - i;
-	current_or_arm = 0;
-	lower_or_arms = upper_or_arms = NIL;
-	need_next_lower_arm = need_next_upper_arm = true;
-	while (current_or_arm < num_or_arms)
-	{
-		List	   *lower_or_arm_args = NIL,
-				   *upper_or_arm_args = NIL;
-
-		/* Restart scan of columns from the i'th one */
-		j = i;
-		partexprs_item = partexprs_item_saved;
-
-		for_both_cell(cell1, lower_or_start_datum, cell2, upper_or_start_datum)
-		{
-			PartitionRangeDatum *ldatum_next = NULL,
-					   *udatum_next = NULL;
-
-			ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
-			if (lnext(cell1))
-				ldatum_next = castNode(PartitionRangeDatum,
-									   lfirst(lnext(cell1)));
-			udatum = castNode(PartitionRangeDatum, lfirst(cell2));
-			if (lnext(cell2))
-				udatum_next = castNode(PartitionRangeDatum,
-									   lfirst(lnext(cell2)));
-			get_range_key_properties(key, j, ldatum, udatum,
-									 &partexprs_item,
-									 &keyCol,
-									 &lower_val, &upper_val);
-
-			if (need_next_lower_arm && lower_val)
-			{
-				uint16		strategy;
-
-				/*
-				 * For the non-last columns of this arm, use the EQ operator.
-				 * For the last column of this arm, use GT, unless this is the
-				 * last column of the whole bound check, or the next bound
-				 * datum is MINVALUE, in which case use GE.
-				 */
-				if (j - i < current_or_arm)
-					strategy = BTEqualStrategyNumber;
-				else if (j == key->partnatts - 1 ||
-						 (ldatum_next &&
-						  ldatum_next->kind == PARTITION_RANGE_DATUM_MINVALUE))
-					strategy = BTGreaterEqualStrategyNumber;
-				else
-					strategy = BTGreaterStrategyNumber;
-
-				lower_or_arm_args = lappend(lower_or_arm_args,
-											make_partition_op_expr(key, j,
-																   strategy,
-																   keyCol,
-																   (Expr *) lower_val));
-			}
-
-			if (need_next_upper_arm && upper_val)
-			{
-				uint16		strategy;
-
-				/*
-				 * For the non-last columns of this arm, use the EQ operator.
-				 * For the last column of this arm, use LT, unless the next
-				 * bound datum is MAXVALUE, in which case use LE.
-				 */
-				if (j - i < current_or_arm)
-					strategy = BTEqualStrategyNumber;
-				else if (udatum_next &&
-						 udatum_next->kind == PARTITION_RANGE_DATUM_MAXVALUE)
-					strategy = BTLessEqualStrategyNumber;
-				else
-					strategy = BTLessStrategyNumber;
-
-				upper_or_arm_args = lappend(upper_or_arm_args,
-											make_partition_op_expr(key, j,
-																   strategy,
-																   keyCol,
-																   (Expr *) upper_val));
-			}
-
-			/*
-			 * Did we generate enough of OR's arguments?  First arm considers
-			 * the first of the remaining columns, second arm considers first
-			 * two of the remaining columns, and so on.
-			 */
-			++j;
-			if (j - i > current_or_arm)
-			{
-				/*
-				 * We must not emit any more arms if the new column that will
-				 * be considered is unbounded, or this one was.
-				 */
-				if (!lower_val || !ldatum_next ||
-					ldatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
-					need_next_lower_arm = false;
-				if (!upper_val || !udatum_next ||
-					udatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
-					need_next_upper_arm = false;
-				break;
-			}
-		}
-
-		if (lower_or_arm_args != NIL)
-			lower_or_arms = lappend(lower_or_arms,
-									list_length(lower_or_arm_args) > 1
-									? makeBoolExpr(AND_EXPR, lower_or_arm_args, -1)
-									: linitial(lower_or_arm_args));
-
-		if (upper_or_arm_args != NIL)
-			upper_or_arms = lappend(upper_or_arms,
-									list_length(upper_or_arm_args) > 1
-									? makeBoolExpr(AND_EXPR, upper_or_arm_args, -1)
-									: linitial(upper_or_arm_args));
-
-		/* If no work to do in the next iteration, break away. */
-		if (!need_next_lower_arm && !need_next_upper_arm)
-			break;
-
-		++current_or_arm;
-	}
-
-	/*
-	 * Generate the OR expressions for each of lower and upper bounds (if
-	 * required), and append to the list of implicitly ANDed list of
-	 * expressions.
-	 */
-	if (lower_or_arms != NIL)
-		result = lappend(result,
-						 list_length(lower_or_arms) > 1
-						 ? makeBoolExpr(OR_EXPR, lower_or_arms, -1)
-						 : linitial(lower_or_arms));
-	if (upper_or_arms != NIL)
-		result = lappend(result,
-						 list_length(upper_or_arms) > 1
-						 ? makeBoolExpr(OR_EXPR, upper_or_arms, -1)
-						 : linitial(upper_or_arms));
-
-	/*
-	 * As noted above, for non-default, we return list with constant TRUE. If
-	 * the result is NIL during the recursive call for default, it implies
-	 * this is the only other partition which can hold every value of the key
-	 * except NULL. Hence we return the NullTest result skipped earlier.
-	 */
-	if (result == NIL)
-		result = for_default
-			? get_range_nulltest(key)
-			: list_make1(makeBoolConst(true, false));
-
-	return result;
-}
-
-/*
- * generate_partition_qual
- *
- * Generate partition predicate from rel's partition bound expression. The
- * function returns a NIL list if there is no predicate.
- *
- * Result expression tree is stored CacheMemoryContext to ensure it survives
- * as long as the relcache entry. But we should be running in a less long-lived
- * working context. To avoid leaking cache memory if this routine fails partway
- * through, we build in working memory and then copy the completed structure
- * into cache memory.
- */
-static List *
-generate_partition_qual(Relation rel)
-{
-	HeapTuple	tuple;
-	MemoryContext oldcxt;
-	Datum		boundDatum;
-	bool		isnull;
-	PartitionBoundSpec *bound;
-	List	   *my_qual = NIL,
-			   *result = NIL;
-	Relation	parent;
-	bool		found_whole_row;
-
-	/* Guard against stack overflow due to overly deep partition tree */
-	check_stack_depth();
-
-	/* Quick copy */
-	if (rel->rd_partcheck != NIL)
-		return copyObject(rel->rd_partcheck);
-
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
-					   AccessShareLock);
-
-	/* Get pg_class.relpartbound */
-	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(rel));
-
-	boundDatum = SysCacheGetAttr(RELOID, tuple,
-								 Anum_pg_class_relpartbound,
-								 &isnull);
-	if (isnull)					/* should not happen */
-		elog(ERROR, "relation \"%s\" has relpartbound = null",
-			 RelationGetRelationName(rel));
-	bound = castNode(PartitionBoundSpec,
-					 stringToNode(TextDatumGetCString(boundDatum)));
-	ReleaseSysCache(tuple);
-
-	my_qual = get_qual_from_partbound(rel, parent, bound);
-
-	/* 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;
-
-	/*
-	 * Change Vars to have partition's attnos instead of the parent's. We do
-	 * this after we concatenate the parent's quals, because we want every Var
-	 * in it to bear this relation's attnos. It's safe to assume varno = 1
-	 * here.
-	 */
-	result = map_partition_varattnos(result, 1, rel, parent,
-									 &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");
-
-	/* Save a copy in the relcache */
-	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
-	rel->rd_partcheck = copyObject(result);
-	MemoryContextSwitchTo(oldcxt);
-
-	/* Keep the parent locked until commit */
-	heap_close(parent, NoLock);
-
-	return result;
-}
-
-/*
- * get_partition_for_tuple
- *		Finds partition of relation which accepts the partition key specified
- *		in values and isnull
- *
- * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
- * found or -1 if none found.
- */
-int
-get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
-{
-	int			bound_offset;
-	int			part_index = -1;
-	PartitionKey key = RelationGetPartitionKey(relation);
-	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
-
-	/* Route as appropriate based on partitioning strategy. */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				PartitionBoundInfo boundinfo = partdesc->boundinfo;
-				int			greatest_modulus = get_greatest_modulus(boundinfo);
-				uint64		rowHash = compute_hash_value(key, values, isnull);
-
-				part_index = boundinfo->indexes[rowHash % greatest_modulus];
-			}
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			if (isnull[0])
-			{
-				if (partition_bound_accepts_nulls(partdesc->boundinfo))
-					part_index = partdesc->boundinfo->null_index;
-			}
-			else
-			{
-				bool		equal = false;
-
-				bound_offset = partition_list_bsearch(key,
-													  partdesc->boundinfo,
-													  values[0], &equal);
-				if (bound_offset >= 0 && equal)
-					part_index = partdesc->boundinfo->indexes[bound_offset];
-			}
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				bool		equal = false,
-							range_partkey_has_null = false;
-				int			i;
-
-				/*
-				 * No range includes NULL, so this will be accepted by the
-				 * default partition if there is one, and otherwise rejected.
-				 */
-				for (i = 0; i < key->partnatts; i++)
-				{
-					if (isnull[i])
-					{
-						range_partkey_has_null = true;
-						break;
-					}
-				}
-
-				if (!range_partkey_has_null)
-				{
-					bound_offset = partition_range_datum_bsearch(key,
-														partdesc->boundinfo,
-														key->partnatts,
-														values,
-														&equal);
-					/*
-					 * The bound at bound_offset is less than or equal to the
-					 * tuple value, so the bound at offset+1 is the upper
-					 * bound of the partition we're looking for, if there
-					 * actually exists one.
-					 */
-					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
-				}
-			}
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/*
-	 * part_index < 0 means we failed to find a partition of this parent. Use
-	 * the default partition, if there is one.
-	 */
-	if (part_index < 0)
-		part_index = partdesc->boundinfo->default_index;
-
-	return part_index;
-}
-
-/*
- * Checks if any of the 'attnums' is a partition key attribute for rel
- *
- * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
- * partition key expression.  It's possible for a column to be both used
- * directly and as part of an expression; if that happens, *used_in_expr may
- * end up as either true or false.  That's OK for current uses of this
- * function, because *used_in_expr is only used to tailor the error message
- * text.
- */
-bool
-has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr)
-{
-	PartitionKey key;
-	int			partnatts;
-	List	   *partexprs;
-	ListCell   *partexprs_item;
-	int			i;
-
-	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return false;
-
-	key = RelationGetPartitionKey(rel);
-	partnatts = get_partition_natts(key);
-	partexprs = get_partition_exprs(key);
-
-	partexprs_item = list_head(partexprs);
-	for (i = 0; i < partnatts; i++)
-	{
-		AttrNumber	partattno = get_partition_col_attnum(key, i);
-
-		if (partattno != 0)
+		if (key->partattrs[i] != 0)
 		{
-			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
-							  attnums))
-			{
-				if (used_in_expr)
-					*used_in_expr = false;
-				return true;
-			}
+			keyCol = (Expr *) makeVar(1,
+									  key->partattrs[i],
+									  key->parttypid[i],
+									  key->parttypmod[i],
+									  key->parttypcoll[i],
+									  0);
 		}
 		else
 		{
-			/* Arbitrary expression */
-			Node	   *expr = (Node *) lfirst(partexprs_item);
-			Bitmapset  *expr_attrs = NULL;
-
-			/* Find all attributes referenced */
-			pull_varattnos(expr, 1, &expr_attrs);
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+			keyCol = copyObject(lfirst(partexprs_item));
 			partexprs_item = lnext(partexprs_item);
-
-			if (bms_overlap(attnums, expr_attrs))
-			{
-				if (used_in_expr)
-					*used_in_expr = true;
-				return true;
-			}
 		}
+
+		nulltest = makeNode(NullTest);
+		nulltest->arg = keyCol;
+		nulltest->nulltesttype = IS_NOT_NULL;
+		nulltest->argisrow = false;
+		nulltest->location = -1;
+		result = lappend(result, nulltest);
 	}
 
-	return false;
+	return result;
 }
 
 /*
- * qsort_partition_hbound_cmp
+ * get_qual_for_range
  *
- * We sort hash bounds by modulus, then by remainder.
- */
-static int32
-qsort_partition_hbound_cmp(const void *a, const void *b)
-{
-	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
-	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
-
-	return partition_hbound_cmp(h1->modulus, h1->remainder,
-								h2->modulus, h2->remainder);
-}
-
-/*
- * partition_hbound_cmp
+ * Returns an implicit-AND list of expressions to use as a range partition's
+ * constraint, given the partition key and bound structures.
  *
- * Compares modulus first, then remainder if modulus are equal.
- */
-static int32
-partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
-{
-	if (modulus1 < modulus2)
-		return -1;
-	if (modulus1 > modulus2)
-		return 1;
-	if (modulus1 == modulus2 && remainder1 != remainder2)
-		return (remainder1 > remainder2) ? 1 : -1;
-	return 0;
-}
-
-/*
- * qsort_partition_list_value_cmp
+ * For a multi-column range partition key, say (a, b, c), with (al, bl, cl)
+ * as the lower bound tuple and (au, bu, cu) as the upper bound tuple, we
+ * generate an expression tree of the following form:
+ *
+ *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
+ *		AND
+ *	(a > al OR (a = al AND b > bl) OR (a = al AND b = bl AND c >= cl))
+ *		AND
+ *	(a < au OR (a = au AND b < bu) OR (a = au AND b = bu AND c < cu))
+ *
+ * It is often the case that a prefix of lower and upper bound tuples contains
+ * the same values, for example, (al = au), in which case, we will emit an
+ * expression tree of the following form:
+ *
+ *	(a IS NOT NULL) and (b IS NOT NULL) and (c IS NOT NULL)
+ *		AND
+ *	(a = al)
+ *		AND
+ *	(b > bl OR (b = bl AND c >= cl))
+ *		AND
+ *	(b < bu) OR (b = bu AND c < cu))
+ *
+ * If a bound datum is either MINVALUE or MAXVALUE, these expressions are
+ * simplified using the fact that any value is greater than MINVALUE and less
+ * than MAXVALUE. So, for example, if cu = MAXVALUE, c < cu is automatically
+ * true, and we need not emit any expression for it, and the last line becomes
+ *
+ *	(b < bu) OR (b = bu), which is simplified to (b <= bu)
+ *
+ * In most common cases with only one partition column, say a, the following
+ * expression tree will be generated: a IS NOT NULL AND a >= al AND a < au
  *
- * Compare two list partition bound datums
+ * For default partition, it returns the negation of the constraints of all
+ * the other partitions.
+ *
+ * External callers should pass for_default as false; we set it to true only
+ * when recursing.
  */
-static int32
-qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+static List *
+get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
+				   bool for_default)
 {
-	Datum		val1 = (*(const PartitionListValue **) a)->value,
-				val2 = (*(const PartitionListValue **) b)->value;
-	PartitionKey key = (PartitionKey) arg;
+	List	   *result = NIL;
+	ListCell   *cell1,
+			   *cell2,
+			   *partexprs_item,
+			   *partexprs_item_saved;
+	int			i,
+				j;
+	PartitionRangeDatum *ldatum,
+			   *udatum;
+	PartitionKey key = RelationGetPartitionKey(parent);
+	Expr	   *keyCol;
+	Const	   *lower_val,
+			   *upper_val;
+	List	   *lower_or_arms,
+			   *upper_or_arms;
+	int			num_or_arms,
+				current_or_arm;
+	ListCell   *lower_or_start_datum,
+			   *upper_or_start_datum;
+	bool		need_next_lower_arm,
+				need_next_upper_arm;
 
-	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-										   key->partcollation[0],
-										   val1, val2));
-}
+	if (spec->is_default)
+	{
+		List	   *or_expr_args = NIL;
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		Oid		   *inhoids = pdesc->oids;
+		int			nparts = pdesc->nparts,
+					i;
 
-/*
- * make_one_range_bound
- *
- * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
- * and a flag telling whether the bound is lower or not.  Made into a function
- * because there are multiple sites that want to use this facility.
- */
-static PartitionRangeBound *
-make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
-{
-	PartitionRangeBound *bound;
-	ListCell   *lc;
-	int			i;
+		for (i = 0; i < nparts; i++)
+		{
+			Oid			inhrelid = inhoids[i];
+			HeapTuple	tuple;
+			Datum		datum;
+			bool		isnull;
+			PartitionBoundSpec *bspec;
 
-	Assert(datums != NIL);
+			tuple = SearchSysCache1(RELOID, inhrelid);
+			if (!HeapTupleIsValid(tuple))
+				elog(ERROR, "cache lookup failed for relation %u", inhrelid);
 
-	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
-	bound->index = index;
-	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
-	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
-													  sizeof(PartitionRangeDatumKind));
-	bound->lower = lower;
+			datum = SysCacheGetAttr(RELOID, tuple,
+									Anum_pg_class_relpartbound,
+									&isnull);
 
-	i = 0;
-	foreach(lc, datums)
-	{
-		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+			Assert(!isnull);
+			bspec = (PartitionBoundSpec *)
+				stringToNode(TextDatumGetCString(datum));
+			if (!IsA(bspec, PartitionBoundSpec))
+				elog(ERROR, "expected PartitionBoundSpec");
+
+			if (!bspec->is_default)
+			{
+				List	   *part_qual;
+
+				part_qual = get_qual_for_range(parent, bspec, true);
 
-		/* What's contained in this range datum? */
-		bound->kind[i] = datum->kind;
+				/*
+				 * AND the constraints of the partition and add to
+				 * or_expr_args
+				 */
+				or_expr_args = lappend(or_expr_args, list_length(part_qual) > 1
+									   ? makeBoolExpr(AND_EXPR, part_qual, -1)
+									   : linitial(part_qual));
+			}
+			ReleaseSysCache(tuple);
+		}
 
-		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		if (or_expr_args != NIL)
 		{
-			Const	   *val = castNode(Const, datum->value);
+			Expr	   *other_parts_constr;
 
-			if (val->constisnull)
-				elog(ERROR, "invalid range bound datum");
-			bound->datums[i] = val->constvalue;
+			/*
+			 * Combine the constraints obtained for non-default partitions
+			 * using OR.  As requested, each of the OR's args doesn't include
+			 * the NOT NULL test for partition keys (which is to avoid its
+			 * useless repetition).  Add the same now.
+			 */
+			other_parts_constr =
+				makeBoolExpr(AND_EXPR,
+							 lappend(get_range_nulltest(key),
+									 list_length(or_expr_args) > 1
+									 ? makeBoolExpr(OR_EXPR, or_expr_args,
+													-1)
+									 : linitial(or_expr_args)),
+							 -1);
+
+			/*
+			 * Finally, the default partition contains everything *NOT*
+			 * contained in the non-default partitions.
+			 */
+			result = list_make1(makeBoolExpr(NOT_EXPR,
+											 list_make1(other_parts_constr), -1));
 		}
 
-		i++;
+		return result;
 	}
 
-	return bound;
-}
+	lower_or_start_datum = list_head(spec->lowerdatums);
+	upper_or_start_datum = list_head(spec->upperdatums);
+	num_or_arms = key->partnatts;
 
-/* Used when sorting range bounds across all range partitions */
-static int32
-qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
-{
-	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
-	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
-	PartitionKey key = (PartitionKey) arg;
+	/*
+	 * If it is the recursive call for default, we skip the get_range_nulltest
+	 * to avoid accumulating the NullTest on the same keys for each partition.
+	 */
+	if (!for_default)
+		result = get_range_nulltest(key);
 
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
-}
+	/*
+	 * Iterate over the key columns and check if the corresponding lower and
+	 * upper datums are equal using the btree equality operator for the
+	 * column's type.  If equal, we emit single keyCol = common_value
+	 * expression.  Starting from the first column for which the corresponding
+	 * lower and upper bound datums are not equal, we generate OR expressions
+	 * as shown in the function's header comment.
+	 */
+	i = 0;
+	partexprs_item = list_head(key->partexprs);
+	partexprs_item_saved = partexprs_item;	/* placate compiler */
+	forboth(cell1, spec->lowerdatums, cell2, spec->upperdatums)
+	{
+		EState	   *estate;
+		MemoryContext oldcxt;
+		Expr	   *test_expr;
+		ExprState  *test_exprstate;
+		Datum		test_result;
+		bool		isNull;
 
-/*
- * partition_rbound_cmp
- *
- * Return for two range bounds whether the 1st one (specified in datums1,
- * kind1, and lower1) is <, =, or > the bound specified in *b2.
- *
- * Note that if the values of the two range bounds compare equal, then we take
- * into account whether they are upper or lower bounds, and an upper bound is
- * considered to be smaller than a lower bound. This is important to the way
- * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
- * structure, which only stores the upper bound of a common boundary between
- * two contiguous partitions.
- */
-static int32
-partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2)
-{
-	int32		cmpval = 0;		/* placate compiler */
-	int			i;
-	Datum	   *datums2 = b2->datums;
-	PartitionRangeDatumKind *kind2 = b2->kind;
-	bool		lower2 = b2->lower;
+		ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
+		udatum = castNode(PartitionRangeDatum, lfirst(cell2));
+
+		/*
+		 * Since get_range_key_properties() modifies partexprs_item, and we
+		 * might need to start over from the previous expression in the later
+		 * part of this function, save away the current value.
+		 */
+		partexprs_item_saved = partexprs_item;
+
+		get_range_key_properties(key, i, ldatum, udatum,
+								 &partexprs_item,
+								 &keyCol,
+								 &lower_val, &upper_val);
+
+		/*
+		 * If either value is NULL, the corresponding partition bound is
+		 * either MINVALUE or MAXVALUE, and we treat them as unequal, because
+		 * even if they're the same, there is no common value to equate the
+		 * key column with.
+		 */
+		if (!lower_val || !upper_val)
+			break;
+
+		/* Create the test expression */
+		estate = CreateExecutorState();
+		oldcxt = MemoryContextSwitchTo(estate->es_query_cxt);
+		test_expr = make_partition_op_expr(key, i, BTEqualStrategyNumber,
+										   (Expr *) lower_val,
+										   (Expr *) upper_val);
+		fix_opfuncids((Node *) test_expr);
+		test_exprstate = ExecInitExpr(test_expr, NULL);
+		test_result = ExecEvalExprSwitchContext(test_exprstate,
+												GetPerTupleExprContext(estate),
+												&isNull);
+		MemoryContextSwitchTo(oldcxt);
+		FreeExecutorState(estate);
+
+		/* If not equal, go generate the OR expressions */
+		if (!DatumGetBool(test_result))
+			break;
 
-	for (i = 0; i < key->partnatts; i++)
-	{
 		/*
-		 * First, handle cases where the column is unbounded, which should not
-		 * invoke the comparison procedure, and should not consider any later
-		 * columns. Note that the PartitionRangeDatumKind enum elements
-		 * compare the same way as the values they represent.
+		 * The bounds for the last key column can't be equal, because such a
+		 * range partition would never be allowed to be defined (it would have
+		 * an empty range otherwise).
 		 */
-		if (kind1[i] < kind2[i])
-			return -1;
-		else if (kind1[i] > kind2[i])
-			return 1;
-		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+		if (i == key->partnatts - 1)
+			elog(ERROR, "invalid range bound specification");
 
-			/*
-			 * The column bounds are both MINVALUE or both MAXVALUE. No later
-			 * columns should be considered, but we still need to compare
-			 * whether they are upper or lower bounds.
-			 */
-			break;
+		/* Equal, so generate keyCol = lower_val expression */
+		result = lappend(result,
+						 make_partition_op_expr(key, i, BTEqualStrategyNumber,
+												keyCol, (Expr *) lower_val));
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 datums1[i],
-												 datums2[i]));
-		if (cmpval != 0)
-			break;
+		i++;
 	}
 
-	/*
-	 * If the comparison is anything other than equal, we're done. If they
-	 * compare equal though, we still have to consider whether the boundaries
-	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
-	 * two.
-	 */
-	if (cmpval == 0 && lower1 != lower2)
-		cmpval = lower1 ? 1 : -1;
-
-	return cmpval;
-}
-
-/*
- * partition_rbound_datum_cmp
- *
- * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
- * is <, =, or > partition key of tuple (tuple_datums)
- */
-static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums)
-{
-	int			i;
-	int32		cmpval = -1;
+	/* First pair of lower_val and upper_val that are not equal. */
+	lower_or_start_datum = cell1;
+	upper_or_start_datum = cell2;
 
-	for (i = 0; i < n_tuple_datums; i++)
+	/* OR will have as many arms as there are key columns left. */
+	num_or_arms = key->partnatts - i;
+	current_or_arm = 0;
+	lower_or_arms = upper_or_arms = NIL;
+	need_next_lower_arm = need_next_upper_arm = true;
+	while (current_or_arm < num_or_arms)
 	{
-		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
-			return -1;
-		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
-			return 1;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 rb_datums[i],
-												 tuple_datums[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	return cmpval;
-}
+		List	   *lower_or_arm_args = NIL,
+				   *upper_or_arm_args = NIL;
 
-/*
- * partition_list_bsearch
- *		Returns the index of the greatest bound datum that is less than equal
- * 		to the given value or -1 if all of the bound datums are greater
- *
- * *is_equal is set to true if the bound datum at the returned index is equal
- * to the input value.
- */
-static int
-partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
+		/* Restart scan of columns from the i'th one */
+		j = i;
+		partexprs_item = partexprs_item_saved;
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-												 key->partcollation[0],
-												 boundinfo->datums[mid][0],
-												 value));
-		if (cmpval <= 0)
+		for_both_cell(cell1, lower_or_start_datum, cell2, upper_or_start_datum)
 		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
+			PartitionRangeDatum *ldatum_next = NULL,
+					   *udatum_next = NULL;
 
-	return lo;
-}
+			ldatum = castNode(PartitionRangeDatum, lfirst(cell1));
+			if (lnext(cell1))
+				ldatum_next = castNode(PartitionRangeDatum,
+									   lfirst(lnext(cell1)));
+			udatum = castNode(PartitionRangeDatum, lfirst(cell2));
+			if (lnext(cell2))
+				udatum_next = castNode(PartitionRangeDatum,
+									   lfirst(lnext(cell2)));
+			get_range_key_properties(key, j, ldatum, udatum,
+									 &partexprs_item,
+									 &keyCol,
+									 &lower_val, &upper_val);
 
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given range bound or -1 if all of the range bounds are
- *		greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input range bound
- */
-static int
-partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
+			if (need_next_lower_arm && lower_val)
+			{
+				uint16		strategy;
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_cmp(key,
-									  boundinfo->datums[mid],
-									  boundinfo->kind[mid],
-									  (boundinfo->indexes[mid] == -1),
-									  probe);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
+				/*
+				 * For the non-last columns of this arm, use the EQ operator.
+				 * For the last column of this arm, use GT, unless this is the
+				 * last column of the whole bound check, or the next bound
+				 * datum is MINVALUE, in which case use GE.
+				 */
+				if (j - i < current_or_arm)
+					strategy = BTEqualStrategyNumber;
+				else if (j == key->partnatts - 1 ||
+						 (ldatum_next &&
+						  ldatum_next->kind == PARTITION_RANGE_DATUM_MINVALUE))
+					strategy = BTGreaterEqualStrategyNumber;
+				else
+					strategy = BTGreaterStrategyNumber;
 
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
+				lower_or_arm_args = lappend(lower_or_arm_args,
+											make_partition_op_expr(key, j,
+																   strategy,
+																   keyCol,
+																   (Expr *) lower_val));
+			}
 
-	return lo;
-}
+			if (need_next_upper_arm && upper_val)
+			{
+				uint16		strategy;
 
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given tuple or -1 if all of the range bounds are greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input tuple.
- */
-static int
-partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
+				/*
+				 * For the non-last columns of this arm, use the EQ operator.
+				 * For the last column of this arm, use LT, unless the next
+				 * bound datum is MAXVALUE, in which case use LE.
+				 */
+				if (j - i < current_or_arm)
+					strategy = BTEqualStrategyNumber;
+				else if (udatum_next &&
+						 udatum_next->kind == PARTITION_RANGE_DATUM_MAXVALUE)
+					strategy = BTLessEqualStrategyNumber;
+				else
+					strategy = BTLessStrategyNumber;
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_datum_cmp(key,
-											boundinfo->datums[mid],
-											boundinfo->kind[mid],
-											values,
-											nvalues);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
+				upper_or_arm_args = lappend(upper_or_arm_args,
+											make_partition_op_expr(key, j,
+																   strategy,
+																   keyCol,
+																   (Expr *) upper_val));
+			}
 
-			if (*is_equal)
+			/*
+			 * Did we generate enough of OR's arguments?  First arm considers
+			 * the first of the remaining columns, second arm considers first
+			 * two of the remaining columns, and so on.
+			 */
+			++j;
+			if (j - i > current_or_arm)
+			{
+				/*
+				 * We must not emit any more arms if the new column that will
+				 * be considered is unbounded, or this one was.
+				 */
+				if (!lower_val || !ldatum_next ||
+					ldatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
+					need_next_lower_arm = false;
+				if (!upper_val || !udatum_next ||
+					udatum_next->kind != PARTITION_RANGE_DATUM_VALUE)
+					need_next_upper_arm = false;
 				break;
+			}
 		}
-		else
-			hi = mid - 1;
-	}
 
-	return lo;
-}
+		if (lower_or_arm_args != NIL)
+			lower_or_arms = lappend(lower_or_arms,
+									list_length(lower_or_arm_args) > 1
+									? makeBoolExpr(AND_EXPR, lower_or_arm_args, -1)
+									: linitial(lower_or_arm_args));
 
-/*
- * partition_hash_bsearch
- *		Returns the index of the greatest (modulus, remainder) pair that is
- *		less than or equal to the given (modulus, remainder) pair or -1 if
- *		all of them are greater
- */
-static int
-partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder)
-{
-	int			lo,
-				hi,
-				mid;
+		if (upper_or_arm_args != NIL)
+			upper_or_arms = lappend(upper_or_arms,
+									list_length(upper_or_arm_args) > 1
+									? makeBoolExpr(AND_EXPR, upper_or_arm_args, -1)
+									: linitial(upper_or_arm_args));
 
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval,
-					bound_modulus,
-					bound_remainder;
-
-		mid = (lo + hi + 1) / 2;
-		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
-		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
-		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
-									  modulus, remainder);
-		if (cmpval <= 0)
-		{
-			lo = mid;
+		/* If no work to do in the next iteration, break away. */
+		if (!need_next_lower_arm && !need_next_upper_arm)
+			break;
 
-			if (cmpval == 0)
-				break;
-		}
-		else
-			hi = mid - 1;
+		++current_or_arm;
 	}
 
-	return lo;
-}
+	/*
+	 * Generate the OR expressions for each of lower and upper bounds (if
+	 * required), and append to the list of implicitly ANDed list of
+	 * expressions.
+	 */
+	if (lower_or_arms != NIL)
+		result = lappend(result,
+						 list_length(lower_or_arms) > 1
+						 ? makeBoolExpr(OR_EXPR, lower_or_arms, -1)
+						 : linitial(lower_or_arms));
+	if (upper_or_arms != NIL)
+		result = lappend(result,
+						 list_length(upper_or_arms) > 1
+						 ? makeBoolExpr(OR_EXPR, upper_or_arms, -1)
+						 : linitial(upper_or_arms));
 
-/*
- * get_default_oid_from_partdesc
- *
- * Given a partition descriptor, return the OID of the default partition, if
- * one exists; else, return InvalidOid.
- */
-Oid
-get_default_oid_from_partdesc(PartitionDesc partdesc)
-{
-	if (partdesc && partdesc->boundinfo &&
-		partition_bound_has_default(partdesc->boundinfo))
-		return partdesc->oids[partdesc->boundinfo->default_index];
+	/*
+	 * As noted above, for non-default, we return list with constant TRUE. If
+	 * the result is NIL during the recursive call for default, it implies
+	 * this is the only other partition which can hold every value of the key
+	 * except NULL. Hence we return the NullTest result skipped earlier.
+	 */
+	if (result == NIL)
+		result = for_default
+			? get_range_nulltest(key)
+			: list_make1(makeBoolConst(true, false));
 
-	return InvalidOid;
+	return result;
 }
 
 /*
@@ -3189,99 +1307,6 @@ get_proposed_default_constraint(List *new_part_constraints)
 }
 
 /*
- * get_partition_bound_num_indexes
- *
- * Returns the number of the entries in the partition bound indexes array.
- */
-static int
-get_partition_bound_num_indexes(PartitionBoundInfo bound)
-{
-	int			num_indexes;
-
-	Assert(bound);
-
-	switch (bound->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-
-			/*
-			 * The number of the entries in the indexes array is same as the
-			 * greatest modulus.
-			 */
-			num_indexes = get_greatest_modulus(bound);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			num_indexes = bound->ndatums;
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			/* Range partitioned table has an extra index. */
-			num_indexes = bound->ndatums + 1;
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) bound->strategy);
-	}
-
-	return num_indexes;
-}
-
-/*
- * get_greatest_modulus
- *
- * Returns the greatest modulus of the hash partition bound. The greatest
- * modulus will be at the end of the datums array because hash partitions are
- * arranged in the ascending order of their modulus and remainders.
- */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
-{
-	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
-	Assert(bound->datums && bound->ndatums > 0);
-	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
-
-	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
-}
-
-/*
- * compute_hash_value
- *
- * Compute the hash value for given not null partition key values.
- */
-static uint64
-compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
-{
-	int			i;
-	int			nkeys = key->partnatts;
-	uint64		rowHash = 0;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-
-	for (i = 0; i < nkeys; i++)
-	{
-		if (!isnull[i])
-		{
-			Datum		hash;
-
-			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
-
-			/*
-			 * Compute hash for each datum value by calling respective
-			 * datatype-specific hash functions of each partition key
-			 * attribute.
-			 */
-			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	return rowHash;
-}
-
-/*
  * satisfies_hash_partition
  *
  * This is an SQL-callable function for use in hash partition constraints.
diff --git a/src/backend/executor/execExpr.c b/src/backend/executor/execExpr.c
index c6eb3ebacf..6c9d89523d 100644
--- a/src/backend/executor/execExpr.c
+++ b/src/backend/executor/execExpr.c
@@ -42,6 +42,7 @@
 #include "optimizer/clauses.h"
 #include "optimizer/planner.h"
 #include "pgstat.h"
+#include "utils/acl.h"
 #include "utils/builtins.h"
 #include "utils/datum.h"
 #include "utils/lsyscache.h"
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 5d3e923cca..b17abb5c7d 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -42,7 +42,6 @@
 #include "access/transam.h"
 #include "access/xact.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_publication.h"
 #include "commands/matview.h"
 #include "commands/trigger.h"
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 4048c3ebc6..cc77ba3701 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -20,6 +20,7 @@
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2a8ecbd830..a4a8326ad3 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -46,6 +46,7 @@
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
+#include "rewrite/rewriteManip.h"
 #include "storage/bufmgr.h"
 #include "storage/lmgr.h"
 #include "utils/builtins.h"
diff --git a/src/backend/libpq/hba.c b/src/backend/libpq/hba.c
index acf625e4ec..f930f62517 100644
--- a/src/backend/libpq/hba.c
+++ b/src/backend/libpq/hba.c
@@ -26,6 +26,7 @@
 #include <unistd.h>
 
 #include "access/htup_details.h"
+#include "catalog/objectaddress.h"
 #include "catalog/pg_collation.h"
 #include "catalog/pg_type.h"
 #include "common/ip.h"
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index f74afdb4dd..7164a09995 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -15,13 +15,13 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/prep.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/memutils.h"
 
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b586f941a8..8c7caabbc7 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,7 +33,6 @@
 #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"
@@ -49,6 +48,7 @@
 #include "parser/parse_coerce.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
 
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 60f21711f4..605493d04a 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,7 +27,6 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_statistic_ext.h"
 #include "foreign/fdwapi.h"
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 5c368321e6..74f8e0a04d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -17,7 +17,6 @@
 #include <limits.h>
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
@@ -28,6 +27,7 @@
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
+#include "utils/partcache.h"
 
 
 typedef struct JoinHashEntry
diff --git a/src/backend/rewrite/rewriteManip.c b/src/backend/rewrite/rewriteManip.c
index abad1bf7e4..c0982a0cfe 100644
--- a/src/backend/rewrite/rewriteManip.c
+++ b/src/backend/rewrite/rewriteManip.c
@@ -13,6 +13,7 @@
  */
 #include "postgres.h"
 
+#include "access/tupconvert.h"
 #include "catalog/pg_type.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -1374,6 +1375,50 @@ map_variable_attnos(Node *node,
 											0);
 }
 
+/*
+ * map_partition_varattnos - maps varattno of any Vars in expr from the
+ * attno's of 'from_rel' to the attno's of 'to_rel' partition, each of which
+ * may be either a leaf partition or a partitioned table, but both of which
+ * must be from the same partitioning hierarchy.
+ *
+ * Even though all of the same column names must be present in all relations
+ * in the hierarchy, and they must also have the same types, the attnos may
+ * be different.
+ *
+ * If found_whole_row is not NULL, *found_whole_row returns whether a
+ * whole-row variable was found in the input expression.
+ *
+ * Note: this will work on any node tree, so really the argument and result
+ * should be declared "Node *".  But a substantial majority of the callers
+ * are working on Lists, so it's less messy to do the casts internally.
+ */
+List *
+map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel,
+						bool *found_whole_row)
+{
+	bool		my_found_whole_row = false;
+
+	if (expr != NIL)
+	{
+		AttrNumber *part_attnos;
+
+		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,
+											fromrel_varno, 0,
+											part_attnos,
+											RelationGetDescr(from_rel)->natts,
+											RelationGetForm(to_rel)->reltype,
+											&my_found_whole_row);
+	}
+
+	if (found_whole_row)
+		*found_whole_row = my_found_whole_row;
+
+	return expr;
+}
 
 /*
  * ReplaceVarsFromTargetList - replace Vars with items from a targetlist
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3abe7d6155..beb3c0c44a 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -67,6 +67,7 @@
 #include "tcop/utility.h"
 #include "utils/acl.h"
 #include "utils/guc.h"
+#include "utils/rel.h"
 #include "utils/syscache.h"
 
 
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 3bb468bdad..107301ebc7 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -24,7 +24,6 @@
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
-#include "catalog/partition.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_authid.h"
diff --git a/src/backend/utils/cache/Makefile b/src/backend/utils/cache/Makefile
index a943f8ea4b..94511eaf54 100644
--- a/src/backend/utils/cache/Makefile
+++ b/src/backend/utils/cache/Makefile
@@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
-	relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
-	typcache.o ts_cache.o
+OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o partcache.o \
+    relcache.o relmapper.o relfilenodemap.o spccache.o syscache.o \
+    lsyscache.o typcache.o ts_cache.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
new file mode 100644
index 0000000000..eb1743475e
--- /dev/null
+++ b/src/backend/utils/cache/partcache.c
@@ -0,0 +1,2115 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.c
+ *		Partitioning related cache data structures and manipulation functions
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *		  src/backend/utils/cache/partcache.c
+ *
+ *-------------------------------------------------------------------------
+*/
+
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "access/nbtree.h"
+#include "access/sysattr.h"
+#include "catalog/partition.h"
+#include "catalog/pg_inherits_fn.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
+#include "utils/builtins.h"
+#include "utils/datum.h"
+#include "utils/hashutils.h"
+#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
+#include "utils/ruleutils.h"
+#include "utils/syscache.h"
+
+/*
+ * When qsort'ing partition bounds after reading from the catalog, each bound
+ * is represented with one of the following structs.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int			modulus;
+	int			remainder;
+	int			index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+	int			index;
+	Datum		value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+	int			index;
+	Datum	   *datums;			/* range bound datums */
+	PartitionRangeDatumKind *kind;	/* the kind of each datum */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+static List *generate_partition_qual(Relation rel);
+
+static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
+					 int remainder2);
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
+static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
+							   void *arg);
+static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
+					 List *datums, bool lower);
+static int32 partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2);
+static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
+						   void *arg);
+
+static int partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal);
+static int partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal);
+static int32 partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums);
+static int partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal);
+static int partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder);
+
+static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
+
+/*
+ * RelationBuildPartitionKey
+ *		Build and attach to relcache partition key data of relation
+ *
+ * Partitioning key data is a complex structure; to avoid complicated logic to
+ * free individual elements whenever the relcache entry is flushed, we give it
+ * its own memory context, child of CacheMemoryContext, which can easily be
+ * deleted on its own.  To avoid leaking memory in that context in case of an
+ * error partway through this function, the context is initially created as a
+ * child of CurTransactionContext and only re-parented to CacheMemoryContext
+ * at the end, when no further errors are possible.  Also, we don't make this
+ * context the current context except in very brief code sections, out of fear
+ * that some of our callees allocate memory on their own which would be leaked
+ * permanently.
+ */
+void
+RelationBuildPartitionKey(Relation relation)
+{
+	Form_pg_partitioned_table form;
+	HeapTuple	tuple;
+	bool		isnull;
+	int			i;
+	PartitionKey key;
+	AttrNumber *attrs;
+	oidvector  *opclass;
+	oidvector  *collation;
+	ListCell   *partexprs_item;
+	Datum		datum;
+	MemoryContext partkeycxt,
+				oldcxt;
+	int16		procnum;
+
+	tuple = SearchSysCache1(PARTRELID,
+							ObjectIdGetDatum(RelationGetRelid(relation)));
+
+	/*
+	 * The following happens when we have created our pg_class entry but not
+	 * the pg_partitioned_table entry yet.
+	 */
+	if (!HeapTupleIsValid(tuple))
+		return;
+
+	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
+											   RelationGetRelationName(relation),
+											   MEMCONTEXT_COPY_NAME,
+											   ALLOCSET_SMALL_SIZES);
+
+	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
+												sizeof(PartitionKeyData));
+
+	/* Fixed-length attributes */
+	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	key->strategy = form->partstrat;
+	key->partnatts = form->partnatts;
+
+	/*
+	 * We can rely on the first variable-length attribute being mapped to the
+	 * relevant field of the catalog's C struct, because all previous
+	 * attributes are non-nullable and fixed-length.
+	 */
+	attrs = form->partattrs.values;
+
+	/* But use the hard way to retrieve further variable-length attributes */
+	/* Operator class */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partclass, &isnull);
+	Assert(!isnull);
+	opclass = (oidvector *) DatumGetPointer(datum);
+
+	/* Collation */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partcollation, &isnull);
+	Assert(!isnull);
+	collation = (oidvector *) DatumGetPointer(datum);
+
+	/* Expressions */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partexprs, &isnull);
+	if (!isnull)
+	{
+		char	   *exprString;
+		Node	   *expr;
+
+		exprString = TextDatumGetCString(datum);
+		expr = stringToNode(exprString);
+		pfree(exprString);
+
+		/*
+		 * Run the expressions through const-simplification since the planner
+		 * will be comparing them to similarly-processed qual clause operands,
+		 * and may fail to detect valid matches without this step; fix
+		 * opfuncids while at it.  We don't need to bother with
+		 * canonicalize_qual() though, because partition expressions are not
+		 * full-fledged qualification clauses.
+		 */
+		expr = eval_const_expressions(NULL, expr);
+		fix_opfuncids(expr);
+
+		oldcxt = MemoryContextSwitchTo(partkeycxt);
+		key->partexprs = (List *) copyObject(expr);
+		MemoryContextSwitchTo(oldcxt);
+	}
+
+	oldcxt = MemoryContextSwitchTo(partkeycxt);
+	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
+	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
+
+	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+
+	/* Gather type and collation info as well */
+	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
+	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
+	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
+	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
+	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	MemoryContextSwitchTo(oldcxt);
+
+	/* determine support function number to search for */
+	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
+		HASHEXTENDED_PROC : BTORDER_PROC;
+
+	/* Copy partattrs and fill other per-attribute info */
+	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		AttrNumber	attno = key->partattrs[i];
+		HeapTuple	opclasstup;
+		Form_pg_opclass opclassform;
+		Oid			funcid;
+
+		/* Collect opfamily information */
+		opclasstup = SearchSysCache1(CLAOID,
+									 ObjectIdGetDatum(opclass->values[i]));
+		if (!HeapTupleIsValid(opclasstup))
+			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
+
+		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
+		key->partopfamily[i] = opclassform->opcfamily;
+		key->partopcintype[i] = opclassform->opcintype;
+
+		/* Get a support function for the specified opfamily and datatypes */
+		funcid = get_opfamily_proc(opclassform->opcfamily,
+								   opclassform->opcintype,
+								   opclassform->opcintype,
+								   procnum);
+		if (!OidIsValid(funcid))
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
+							NameStr(opclassform->opcname),
+							(key->strategy == PARTITION_STRATEGY_HASH) ?
+							"hash" : "btree",
+							procnum,
+							format_type_be(opclassform->opcintype))));
+
+		fmgr_info(funcid, &key->partsupfunc[i]);
+
+		/* Collation */
+		key->partcollation[i] = collation->values[i];
+
+		/* Collect type information */
+		if (attno != 0)
+		{
+			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
+
+			key->parttypid[i] = att->atttypid;
+			key->parttypmod[i] = att->atttypmod;
+			key->parttypcoll[i] = att->attcollation;
+		}
+		else
+		{
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			key->parttypid[i] = exprType(lfirst(partexprs_item));
+			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
+			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
+
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(key->parttypid[i],
+							 &key->parttyplen[i],
+							 &key->parttypbyval[i],
+							 &key->parttypalign[i]);
+
+		ReleaseSysCache(opclasstup);
+	}
+
+	ReleaseSysCache(tuple);
+
+	/*
+	 * Success --- reparent our context and make the relcache point to the
+	 * newly constructed key
+	 */
+	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
+	relation->rd_partkeycxt = partkeycxt;
+	relation->rd_partkey = key;
+}
+
+/*
+ * RelationBuildPartitionDesc
+ *		Form rel's partition descriptor
+ *
+ * Not flushed from the cache by RelationClearRelation() unless changed because
+ * of addition or removal of partition.
+ */
+void
+RelationBuildPartitionDesc(Relation rel)
+{
+	List	   *inhoids,
+			   *partoids;
+	Oid		   *oids = NULL;
+	List	   *boundspecs = NIL;
+	ListCell   *cell;
+	int			i,
+				nparts;
+	PartitionKey key = RelationGetPartitionKey(rel);
+	PartitionDesc result;
+	MemoryContext oldcxt;
+
+	int			ndatums = 0;
+	int			default_index = -1;
+
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
+	/* List partitioning specific */
+	PartitionListValue **all_values = NULL;
+	int			null_index = -1;
+
+	/* Range partitioning specific */
+	PartitionRangeBound **rbounds = NULL;
+
+	/*
+	 * The following could happen in situations where rel has a pg_class entry
+	 * but not the pg_partitioned_table entry yet.
+	 */
+	if (key == NULL)
+		return;
+
+	/* Get partition oids from pg_inherits */
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+
+	/* Collect bound spec nodes in a list */
+	i = 0;
+	partoids = NIL;
+	foreach(cell, inhoids)
+	{
+		Oid			inhrelid = lfirst_oid(cell);
+		HeapTuple	tuple;
+		Datum		datum;
+		bool		isnull;
+		Node	   *boundspec;
+
+		tuple = SearchSysCache1(RELOID, inhrelid);
+		if (!HeapTupleIsValid(tuple))
+			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+		/*
+		 * It is possible that the pg_class tuple of a partition has not been
+		 * updated yet to set its relpartbound field.  The only case where
+		 * this happens is when we open the parent relation to check using its
+		 * partition descriptor that a new partition's bound does not overlap
+		 * some existing partition.
+		 */
+		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		{
+			ReleaseSysCache(tuple);
+			continue;
+		}
+
+		datum = SysCacheGetAttr(RELOID, tuple,
+								Anum_pg_class_relpartbound,
+								&isnull);
+		Assert(!isnull);
+		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+		/*
+		 * Sanity check: If the PartitionBoundSpec says this is the default
+		 * partition, its OID should correspond to whatever's stored in
+		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 */
+		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		{
+			Oid			partdefid;
+
+			partdefid = get_default_partition_oid(RelationGetRelid(rel));
+			if (partdefid != inhrelid)
+				elog(ERROR, "expected partdefid %u, but got %u",
+					 inhrelid, partdefid);
+		}
+
+		boundspecs = lappend(boundspecs, boundspec);
+		partoids = lappend_oid(partoids, inhrelid);
+		ReleaseSysCache(tuple);
+	}
+
+	nparts = list_length(partoids);
+
+	if (nparts > 0)
+	{
+		oids = (Oid *) palloc(nparts * sizeof(Oid));
+		i = 0;
+		foreach(cell, partoids)
+			oids[i++] = lfirst_oid(cell);
+
+		/* Convert from node to the internal representation */
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **)
+				palloc(nparts * sizeof(PartitionHashBound *));
+
+			i = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
+		{
+			List	   *non_null_values = NIL;
+
+			/*
+			 * Create a unified list of non-null values across all partitions.
+			 */
+			i = 0;
+			null_index = -1;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				ListCell   *c;
+
+				if (spec->strategy != PARTITION_STRATEGY_LIST)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
+				foreach(c, spec->listdatums)
+				{
+					Const	   *val = castNode(Const, lfirst(c));
+					PartitionListValue *list_value = NULL;
+
+					if (!val->constisnull)
+					{
+						list_value = (PartitionListValue *)
+							palloc0(sizeof(PartitionListValue));
+						list_value->index = i;
+						list_value->value = val->constvalue;
+					}
+					else
+					{
+						/*
+						 * Never put a null into the values array, flag
+						 * instead for the code further down below where we
+						 * construct the actual relcache struct.
+						 */
+						if (null_index != -1)
+							elog(ERROR, "found null more than once");
+						null_index = i;
+					}
+
+					if (list_value)
+						non_null_values = lappend(non_null_values,
+												  list_value);
+				}
+
+				i++;
+			}
+
+			ndatums = list_length(non_null_values);
+
+			/*
+			 * Collect all list values in one array. Alongside the value, we
+			 * also save the index of partition the value comes from.
+			 */
+			all_values = (PartitionListValue **) palloc(ndatums *
+														sizeof(PartitionListValue *));
+			i = 0;
+			foreach(cell, non_null_values)
+			{
+				PartitionListValue *src = lfirst(cell);
+
+				all_values[i] = (PartitionListValue *)
+					palloc(sizeof(PartitionListValue));
+				all_values[i]->value = src->value;
+				all_values[i]->index = src->index;
+				i++;
+			}
+
+			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
+					  qsort_partition_list_value_cmp, (void *) key);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_RANGE)
+		{
+			int			k;
+			PartitionRangeBound **all_bounds,
+					   *prev;
+
+			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
+														  sizeof(PartitionRangeBound *));
+
+			/*
+			 * Create a unified list of range bounds across all the
+			 * partitions.
+			 */
+			i = ndatums = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				PartitionRangeBound *lower,
+						   *upper;
+
+				if (spec->strategy != PARTITION_STRATEGY_RANGE)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the allbounds array
+				 * for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i++;
+					continue;
+				}
+
+				lower = make_one_range_bound(key, i, spec->lowerdatums,
+											 true);
+				upper = make_one_range_bound(key, i, spec->upperdatums,
+											 false);
+				all_bounds[ndatums++] = lower;
+				all_bounds[ndatums++] = upper;
+				i++;
+			}
+
+			Assert(ndatums == nparts * 2 ||
+				   (default_index != -1 && ndatums == (nparts - 1) * 2));
+
+			/* Sort all the bounds in ascending order */
+			qsort_arg(all_bounds, ndatums,
+					  sizeof(PartitionRangeBound *),
+					  qsort_partition_rbound_cmp,
+					  (void *) key);
+
+			/* Save distinct bounds from all_bounds into rbounds. */
+			rbounds = (PartitionRangeBound **)
+				palloc(ndatums * sizeof(PartitionRangeBound *));
+			k = 0;
+			prev = NULL;
+			for (i = 0; i < ndatums; i++)
+			{
+				PartitionRangeBound *cur = all_bounds[i];
+				bool		is_distinct = false;
+				int			j;
+
+				/* Is the current bound distinct from the previous one? */
+				for (j = 0; j < key->partnatts; j++)
+				{
+					Datum		cmpval;
+
+					if (prev == NULL || cur->kind[j] != prev->kind[j])
+					{
+						is_distinct = true;
+						break;
+					}
+
+					/*
+					 * If the bounds are both MINVALUE or MAXVALUE, stop now
+					 * and treat them as equal, since any values after this
+					 * point must be ignored.
+					 */
+					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
+						break;
+
+					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
+											   key->partcollation[j],
+											   cur->datums[j],
+											   prev->datums[j]);
+					if (DatumGetInt32(cmpval) != 0)
+					{
+						is_distinct = true;
+						break;
+					}
+				}
+
+				/*
+				 * Only if the bound is distinct save it into a temporary
+				 * array i.e. rbounds which is later copied into boundinfo
+				 * datums array.
+				 */
+				if (is_distinct)
+					rbounds[k++] = all_bounds[i];
+
+				prev = cur;
+			}
+
+			/* Update ndatums to hold the count of distinct datums. */
+			ndatums = k;
+		}
+		else
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/* Now build the actual relcache partition descriptor */
+	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
+												  RelationGetRelationName(rel),
+												  MEMCONTEXT_COPY_NAME,
+												  ALLOCSET_DEFAULT_SIZES);
+	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+
+	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
+	result->nparts = nparts;
+	if (nparts > 0)
+	{
+		PartitionBoundInfo boundinfo;
+		int		   *mapping;
+		int			next_index = 0;
+
+		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
+
+		boundinfo = (PartitionBoundInfoData *)
+			palloc0(sizeof(PartitionBoundInfoData));
+		boundinfo->strategy = key->strategy;
+		boundinfo->default_index = -1;
+		boundinfo->ndatums = ndatums;
+		boundinfo->null_index = -1;
+		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
+
+		/* Initialize mapping array with invalid values */
+		mapping = (int *) palloc(sizeof(int) * nparts);
+		for (i = 0; i < nparts; i++)
+			mapping[i] = -1;
+
+		switch (key->strategy)
+		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+
+					for (i = 0; i < greatest_modulus; i++)
+						boundinfo->indexes[i] = -1;
+
+					for (i = 0; i < nparts; i++)
+					{
+						int			modulus = hbounds[i]->modulus;
+						int			remainder = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(modulus);
+						boundinfo->datums[i][1] = Int32GetDatum(remainder);
+
+						while (remainder < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[remainder] == -1);
+							boundinfo->indexes[remainder] = i;
+							remainder += modulus;
+						}
+
+						mapping[hbounds[i]->index] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
+			case PARTITION_STRATEGY_LIST:
+				{
+					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
+
+					/*
+					 * Copy values.  Indexes of individual values are mapped
+					 * to canonical values so that they match for any two list
+					 * partitioned tables with same number of partitions and
+					 * same lists per partition.  One way to canonicalize is
+					 * to assign the index in all_values[] of the smallest
+					 * value of each partition, as the index of all of the
+					 * partition's values.
+					 */
+					for (i = 0; i < ndatums; i++)
+					{
+						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
+						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
+															key->parttypbyval[0],
+															key->parttyplen[0]);
+
+						/* If the old index has no mapping, assign one */
+						if (mapping[all_values[i]->index] == -1)
+							mapping[all_values[i]->index] = next_index++;
+
+						boundinfo->indexes[i] = mapping[all_values[i]->index];
+					}
+
+					/*
+					 * If null-accepting partition has no mapped index yet,
+					 * assign one.  This could happen if such partition
+					 * accepts only null and hence not covered in the above
+					 * loop which only handled non-null values.
+					 */
+					if (null_index != -1)
+					{
+						Assert(null_index >= 0);
+						if (mapping[null_index] == -1)
+							mapping[null_index] = next_index++;
+						boundinfo->null_index = mapping[null_index];
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any value not
+						 * specified in the lists of other partitions, hence
+						 * it should not get mapped index while assigning
+						 * those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+
+					/* All partition must now have a valid mapping */
+					Assert(next_index == nparts);
+					break;
+				}
+
+			case PARTITION_STRATEGY_RANGE:
+				{
+					boundinfo->kind = (PartitionRangeDatumKind **)
+						palloc(ndatums *
+							   sizeof(PartitionRangeDatumKind *));
+					boundinfo->indexes = (int *) palloc((ndatums + 1) *
+														sizeof(int));
+
+					for (i = 0; i < ndatums; i++)
+					{
+						int			j;
+
+						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
+																sizeof(Datum));
+						boundinfo->kind[i] = (PartitionRangeDatumKind *)
+							palloc(key->partnatts *
+								   sizeof(PartitionRangeDatumKind));
+						for (j = 0; j < key->partnatts; j++)
+						{
+							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
+								boundinfo->datums[i][j] =
+									datumCopy(rbounds[i]->datums[j],
+											  key->parttypbyval[j],
+											  key->parttyplen[j]);
+							boundinfo->kind[i][j] = rbounds[i]->kind[j];
+						}
+
+						/*
+						 * There is no mapping for invalid indexes.
+						 *
+						 * Any lower bounds in the rbounds array have invalid
+						 * indexes assigned, because the values between the
+						 * previous bound (if there is one) and this (lower)
+						 * bound are not part of the range of any existing
+						 * partition.
+						 */
+						if (rbounds[i]->lower)
+							boundinfo->indexes[i] = -1;
+						else
+						{
+							int			orig_index = rbounds[i]->index;
+
+							/* If the old index has no mapping, assign one */
+							if (mapping[orig_index] == -1)
+								mapping[orig_index] = next_index++;
+
+							boundinfo->indexes[i] = mapping[orig_index];
+						}
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						Assert(default_index >= 0 && mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					boundinfo->indexes[i] = -1;
+					break;
+				}
+
+			default:
+				elog(ERROR, "unexpected partition strategy: %d",
+					 (int) key->strategy);
+		}
+
+		result->boundinfo = boundinfo;
+
+		/*
+		 * Now assign OIDs from the original array into mapped indexes of the
+		 * result array.  Order of OIDs in the former is defined by the
+		 * catalog scan that retrieved them, whereas that in the latter is
+		 * defined by canonicalized representation of the partition bounds.
+		 */
+		for (i = 0; i < nparts; i++)
+			result->oids[mapping[i]] = oids[i];
+		pfree(mapping);
+	}
+
+	MemoryContextSwitchTo(oldcxt);
+	rel->rd_partdesc = result;
+}
+
+/*
+ * Are two partition bound collections logically equal?
+ *
+ * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
+ * This is also useful when b1 and b2 are bound collections of two separate
+ * relations, respectively, because PartitionBoundInfo is a canonical
+ * representation of partition bounds.
+ */
+bool
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
+					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			i;
+
+	if (b1->strategy != b2->strategy)
+		return false;
+
+	if (b1->ndatums != b2->ndatums)
+		return false;
+
+	if (b1->null_index != b2->null_index)
+		return false;
+
+	if (b1->default_index != b2->default_index)
+		return false;
+
+	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	{
+		int			greatest_modulus = get_greatest_modulus(b1);
+
+		/*
+		 * If two hash partitioned tables have different greatest moduli,
+		 * their partition schemes don't match.
+		 */
+		if (greatest_modulus != get_greatest_modulus(b2))
+			return false;
+
+		/*
+		 * We arrange the partitions in the ascending order of their modulus
+		 * and remainders.  Also every modulus is factor of next larger
+		 * modulus.  Therefore we can safely store index of a given partition
+		 * in indexes array at remainder of that partition.  Also entries at
+		 * (remainder + N * modulus) positions in indexes array are all same
+		 * for (modulus, remainder) specification for any partition.  Thus
+		 * datums array from both the given bounds are same, if and only if
+		 * their indexes array will be same.  So, it suffices to compare
+		 * indexes array.
+		 */
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Nonetheless make sure that the bounds are indeed same when the
+		 * indexes match.  Hash partition bound stores modulus and remainder
+		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			Assert((b1->datums[i][0] == b2->datums[i][0] &&
+					b1->datums[i][1] == b2->datums[i][1]));
+#endif
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
+		{
+			int			j;
+
+			for (j = 0; j < partnatts; j++)
+			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->kind != NULL)
+				{
+					/* The different kinds of bound all differ from each other */
+					if (b1->kind[i][j] != b2->kind[i][j])
+						return false;
+
+					/*
+					 * Non-finite bounds are equal without further
+					 * examination.
+					 */
+					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
+						continue;
+				}
+
+				/*
+				 * Compare the actual values. Note that it would be both
+				 * incorrect and unsafe to invoke the comparison operator
+				 * derived from the partitioning specification here.  It would
+				 * be incorrect because we want the relcache entry to be
+				 * updated for ANY change to the partition bounds, not just
+				 * those that the partitioning operator thinks are
+				 * significant.  It would be unsafe because we might reach
+				 * this code in the context of an aborted transaction, and an
+				 * arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be
+				 * safe.
+				 */
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  parttypbyval[j], parttyplen[j]))
+					return false;
+			}
+
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+		}
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
+			return false;
+	}
+	return true;
+}
+
+/*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specification.
+ */
+PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo dest;
+	int			i;
+	int			ndatums;
+	int			partnatts;
+	int			num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	num_indexes = get_partition_bound_num_indexes(src);
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+														 sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+															   sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int			j;
+
+		/*
+		 * For a corresponding to hash partition, datums array will have two
+		 * elements - modulus and remainder.
+		 */
+		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
+		int			natts = hash_part ? 2 : partnatts;
+
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+
+		for (j = 0; j < natts; j++)
+		{
+			bool		byval;
+			int			typlen;
+
+			if (hash_part)
+			{
+				typlen = sizeof(int32); /* Always int4 */
+				byval = true;	/* int4 is pass-by-value */
+			}
+			else
+			{
+				byval = key->parttypbyval[j];
+				typlen = key->parttyplen[j];
+			}
+
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   byval, typlen);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
+ * check_new_partition_bound
+ *
+ * Checks if the new partition's bound overlaps any of the existing partitions
+ * of parent.  Also performs additional checks as necessary per strategy.
+ */
+void
+check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
+	ParseState *pstate = make_parsestate(NULL);
+	int			with = -1;
+	bool		overlap = false;
+
+	if (spec->is_default)
+	{
+		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+			return;
+
+		/* Default partition already exists, error out. */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+				 parser_errposition(pstate, spec->location)));
+	}
+
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					Datum	  **datums = boundinfo->datums;
+					int			ndatums = boundinfo->ndatums;
+					int			greatest_modulus;
+					int			remainder;
+					int			offset;
+					bool		valid_modulus = true;
+					int			prev_modulus,	/* Previous largest modulus */
+								next_modulus;	/* Next largest modulus */
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.
+					 *
+					 * Get the greatest (modulus, remainder) pair contained in
+					 * boundinfo->datums that is less than or equal to the
+					 * (spec->modulus, spec->remainder) pair.
+					 */
+					offset = partition_hash_bsearch(key, boundinfo,
+													spec->modulus,
+													spec->remainder);
+					if (offset < 0)
+					{
+						next_modulus = DatumGetInt32(datums[0][0]);
+						valid_modulus = (next_modulus % spec->modulus) == 0;
+					}
+					else
+					{
+						prev_modulus = DatumGetInt32(datums[offset][0]);
+						valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+						if (valid_modulus && (offset + 1) < ndatums)
+						{
+							next_modulus = DatumGetInt32(datums[offset + 1][0]);
+							valid_modulus = (next_modulus % spec->modulus) == 0;
+						}
+					}
+
+					if (!valid_modulus)
+						ereport(ERROR,
+								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
+
+					greatest_modulus = get_greatest_modulus(boundinfo);
+					remainder = spec->remainder;
+
+					/*
+					 * Normally, the lowest remainder that could conflict with
+					 * the new partition is equal to the remainder specified
+					 * for the new partition, but when the new partition has a
+					 * modulus higher than any used so far, we need to adjust.
+					 */
+					if (remainder >= greatest_modulus)
+						remainder = remainder % greatest_modulus;
+
+					/* Check every potentially-conflicting remainder. */
+					do
+					{
+						if (boundinfo->indexes[remainder] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[remainder];
+							break;
+						}
+						remainder += spec->modulus;
+					} while (remainder < greatest_modulus);
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_LIST:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+
+				if (partdesc->nparts > 0)
+				{
+					ListCell   *cell;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					foreach(cell, spec->listdatums)
+					{
+						Const	   *val = castNode(Const, lfirst(cell));
+
+						if (!val->constisnull)
+						{
+							int			offset;
+							bool		equal;
+
+							offset = partition_list_bsearch(key, boundinfo,
+															val->constvalue,
+															&equal);
+							if (offset >= 0 && equal)
+							{
+								overlap = true;
+								with = boundinfo->indexes[offset];
+								break;
+							}
+						}
+						else if (partition_bound_accepts_nulls(boundinfo))
+						{
+							overlap = true;
+							with = boundinfo->null_index;
+							break;
+						}
+					}
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				PartitionRangeBound *lower,
+						   *upper;
+
+				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
+				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
+
+				/*
+				 * First check if the resulting range would be empty with
+				 * specified lower and upper bounds
+				 */
+				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
+										 upper) >= 0)
+				{
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("empty range bound specified for partition \"%s\"",
+									relname),
+							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
+									   get_range_partbound_string(spec->lowerdatums),
+									   get_range_partbound_string(spec->upperdatums)),
+							 parser_errposition(pstate, spec->location)));
+				}
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					int			offset;
+					bool		equal;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Test whether the new lower bound (which is treated
+					 * inclusively as part of the new partition) lies inside
+					 * an existing partition, or in a gap.
+					 *
+					 * If it's inside an existing partition, the bound at
+					 * offset + 1 will be the upper bound of that partition,
+					 * and its index will be >= 0.
+					 *
+					 * If it's in a gap, the bound at offset + 1 will be the
+					 * lower bound of the next partition, and its index will
+					 * be -1. This is also true if there is no next partition,
+					 * since the index array is initialised with an extra -1
+					 * at the end.
+					 */
+					offset = partition_range_bsearch(key, boundinfo, lower,
+													 &equal);
+
+					if (boundinfo->indexes[offset + 1] < 0)
+					{
+						/*
+						 * Check that the new partition will fit in the gap.
+						 * For it to fit, the new upper bound must be less
+						 * than or equal to the lower bound of the next
+						 * partition, if there is one.
+						 */
+						if (offset + 1 < boundinfo->ndatums)
+						{
+							int32		cmpval;
+							Datum 	   *datums;
+							PartitionRangeDatumKind *kind;
+							bool		is_lower;
+
+							datums = boundinfo->datums[offset + 1];
+							kind = boundinfo->kind[offset + 1];
+							is_lower = (boundinfo->indexes[offset + 1] == -1);
+
+							cmpval = partition_rbound_cmp(key, datums, kind,
+														  is_lower, upper);
+							if (cmpval < 0)
+							{
+								/*
+								 * The new partition overlaps with the
+								 * existing partition between offset + 1 and
+								 * offset + 2.
+								 */
+								overlap = true;
+								with = boundinfo->indexes[offset + 2];
+							}
+						}
+					}
+					else
+					{
+						/*
+						 * The new partition overlaps with the existing
+						 * partition between offset and offset + 1.
+						 */
+						overlap = true;
+						with = boundinfo->indexes[offset + 1];
+					}
+				}
+
+				break;
+			}
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	if (overlap)
+	{
+		Assert(with >= 0);
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" would overlap partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[with])),
+				 parser_errposition(pstate, spec->location)));
+	}
+}
+
+/*
+ * RelationGetPartitionQual
+ *
+ * Returns a list of partition quals
+ */
+List *
+RelationGetPartitionQual(Relation rel)
+{
+	/* Quick exit */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
+	return generate_partition_qual(rel);
+}
+
+/*
+ * get_partition_qual_relid
+ *
+ * Returns an expression tree describing the passed-in relation's partition
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
+ */
+Expr *
+get_partition_qual_relid(Oid relid)
+{
+	Relation	rel = heap_open(relid, AccessShareLock);
+	Expr	   *result = NULL;
+	List	   *and_args;
+
+	/* Do the work only if this relation is a partition. */
+	if (rel->rd_rel->relispartition)
+	{
+		and_args = generate_partition_qual(rel);
+
+		if (and_args == NIL)
+			result = NULL;
+		else if (list_length(and_args) > 1)
+			result = makeBoolExpr(AND_EXPR, and_args, -1);
+		else
+			result = linitial(and_args);
+	}
+
+	/* Keep the lock. */
+	heap_close(rel, NoLock);
+
+	return result;
+}
+
+/*
+ * Checks if any of the 'attnums' is a partition key attribute for rel
+ *
+ * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
+ * partition key expression.  It's possible for a column to be both used
+ * directly and as part of an expression; if that happens, *used_in_expr may
+ * end up as either true or false.  That's OK for current uses of this
+ * function, because *used_in_expr is only used to tailor the error message
+ * text.
+ */
+bool
+has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr)
+{
+	PartitionKey key;
+	int			partnatts;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	int			i;
+
+	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		return false;
+
+	key = RelationGetPartitionKey(rel);
+	partnatts = get_partition_natts(key);
+	partexprs = get_partition_exprs(key);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+		{
+			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
+							  attnums))
+			{
+				if (used_in_expr)
+					*used_in_expr = false;
+				return true;
+			}
+		}
+		else
+		{
+			/* Arbitrary expression */
+			Node	   *expr = (Node *) lfirst(partexprs_item);
+			Bitmapset  *expr_attrs = NULL;
+
+			/* Find all attributes referenced */
+			pull_varattnos(expr, 1, &expr_attrs);
+			partexprs_item = lnext(partexprs_item);
+
+			if (bms_overlap(attnums, expr_attrs))
+			{
+				if (used_in_expr)
+					*used_in_expr = true;
+				return true;
+			}
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple
+ *		Finds partition of relation which accepts the partition key specified
+ *		in values and isnull
+ *
+ * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
+ * found or -1 if none found.
+ */
+int
+get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
+{
+	int			bound_offset;
+	int			part_index = -1;
+	PartitionKey key = RelationGetPartitionKey(relation);
+	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
+
+	/* Route as appropriate based on partitioning strategy. */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundInfo boundinfo = partdesc->boundinfo;
+				int			greatest_modulus = get_greatest_modulus(boundinfo);
+				uint64		rowHash = compute_hash_value(key, values, isnull);
+
+				part_index = boundinfo->indexes[rowHash % greatest_modulus];
+			}
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			if (isnull[0])
+			{
+				if (partition_bound_accepts_nulls(partdesc->boundinfo))
+					part_index = partdesc->boundinfo->null_index;
+			}
+			else
+			{
+				bool		equal = false;
+
+				bound_offset = partition_list_bsearch(key,
+													  partdesc->boundinfo,
+													  values[0], &equal);
+				if (bound_offset >= 0 && equal)
+					part_index = partdesc->boundinfo->indexes[bound_offset];
+			}
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				bool		equal = false,
+							range_partkey_has_null = false;
+				int			i;
+
+				/*
+				 * No range includes NULL, so this will be accepted by the
+				 * default partition if there is one, and otherwise rejected.
+				 */
+				for (i = 0; i < key->partnatts; i++)
+				{
+					if (isnull[i])
+					{
+						range_partkey_has_null = true;
+						break;
+					}
+				}
+
+				if (!range_partkey_has_null)
+				{
+					bound_offset = partition_range_datum_bsearch(key,
+														partdesc->boundinfo,
+														key->partnatts,
+														values,
+														&equal);
+					/*
+					 * The bound at bound_offset is less than or equal to the
+					 * tuple value, so the bound at offset+1 is the upper
+					 * bound of the partition we're looking for, if there
+					 * actually exists one.
+					 */
+					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
+				}
+			}
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/*
+	 * part_index < 0 means we failed to find a partition of this parent. Use
+	 * the default partition, if there is one.
+	 */
+	if (part_index < 0)
+		part_index = partdesc->boundinfo->default_index;
+
+	return part_index;
+}
+
+/*
+ * get_greatest_modulus
+ *
+ * Returns the greatest modulus of the hash partition bound. The greatest
+ * modulus will be at the end of the datums array because hash partitions are
+ * arranged in the ascending order of their modulus and remainders.
+ */
+int
+get_greatest_modulus(PartitionBoundInfo bound)
+{
+	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
+	Assert(bound->datums && bound->ndatums > 0);
+	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
+
+	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+uint64
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+	int			i;
+	int			nkeys = key->partnatts;
+	uint64		rowHash = 0;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+	for (i = 0; i < nkeys; i++)
+	{
+		if (!isnull[i])
+		{
+			Datum		hash;
+
+			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
+
+			/*
+			 * Compute hash for each datum value by calling respective
+			 * datatype-specific hash functions of each partition key
+			 * attribute.
+			 */
+			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+	if (partdesc && partdesc->boundinfo &&
+		partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
+
+/* Module-local functions. */
+
+/*
+ * generate_partition_qual
+ *
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
+ *
+ * Result expression tree is stored CacheMemoryContext to ensure it survives
+ * as long as the relcache entry. But we should be running in a less long-lived
+ * working context. To avoid leaking cache memory if this routine fails partway
+ * through, we build in working memory and then copy the completed structure
+ * into cache memory.
+ */
+static List *
+generate_partition_qual(Relation rel)
+{
+	HeapTuple	tuple;
+	MemoryContext oldcxt;
+	Datum		boundDatum;
+	bool		isnull;
+	PartitionBoundSpec *bound;
+	List	   *my_qual = NIL,
+			   *result = NIL;
+	Relation	parent;
+	bool		found_whole_row;
+
+	/* Guard against stack overflow due to overly deep partition tree */
+	check_stack_depth();
+
+	/* Quick copy */
+	if (rel->rd_partcheck != NIL)
+		return copyObject(rel->rd_partcheck);
+
+	/* Grab at least an AccessShareLock on the parent table */
+	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
+					   AccessShareLock);
+
+	/* Get pg_class.relpartbound */
+	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	boundDatum = SysCacheGetAttr(RELOID, tuple,
+								 Anum_pg_class_relpartbound,
+								 &isnull);
+	if (isnull)					/* should not happen */
+		elog(ERROR, "relation \"%s\" has relpartbound = null",
+			 RelationGetRelationName(rel));
+	bound = castNode(PartitionBoundSpec,
+					 stringToNode(TextDatumGetCString(boundDatum)));
+	ReleaseSysCache(tuple);
+
+	my_qual = get_qual_from_partbound(rel, parent, bound);
+
+	/* 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;
+
+	/*
+	 * Change Vars to have partition's attnos instead of the parent's. We do
+	 * this after we concatenate the parent's quals, because we want every Var
+	 * in it to bear this relation's attnos. It's safe to assume varno = 1
+	 * here.
+	 */
+	result = map_partition_varattnos(result, 1, rel, parent,
+									 &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");
+
+	/* Save a copy in the relcache */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	rel->rd_partcheck = copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* Keep the parent locked until commit */
+	heap_close(parent, NoLock);
+
+	return result;
+}
+
+/*
+ * partition_hbound_cmp
+ *
+ * Compares modulus first, then remainder if modulus are equal.
+ */
+static int32
+partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
+{
+	if (modulus1 < modulus2)
+		return -1;
+	if (modulus1 > modulus2)
+		return 1;
+	if (modulus1 == modulus2 && remainder1 != remainder2)
+		return (remainder1 > remainder2) ? 1 : -1;
+	return 0;
+}
+
+/*
+ * qsort_partition_hbound_cmp
+ *
+ * We sort hash bounds by modulus, then by remainder.
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+
+	return partition_hbound_cmp(h1->modulus, h1->remainder,
+								h2->modulus, h2->remainder);
+}
+
+/*
+ * qsort_partition_list_value_cmp
+ *
+ * Compare two list partition bound datums
+ */
+static int32
+qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+{
+	Datum		val1 = (*(const PartitionListValue **) a)->value,
+				val2 = (*(const PartitionListValue **) b)->value;
+	PartitionKey key = (PartitionKey) arg;
+
+	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+										   key->partcollation[0],
+										   val1, val2));
+}
+
+/*
+ * make_one_range_bound
+ *
+ * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
+ * and a flag telling whether the bound is lower or not.  Made into a function
+ * because there are multiple sites that want to use this facility.
+ */
+static PartitionRangeBound *
+make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
+{
+	PartitionRangeBound *bound;
+	ListCell   *lc;
+	int			i;
+
+	Assert(datums != NIL);
+
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
+	bound->index = index;
+	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
+	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
+													  sizeof(PartitionRangeDatumKind));
+	bound->lower = lower;
+
+	i = 0;
+	foreach(lc, datums)
+	{
+		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+
+		/* What's contained in this range datum? */
+		bound->kind[i] = datum->kind;
+
+		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		{
+			Const	   *val = castNode(Const, datum->value);
+
+			if (val->constisnull)
+				elog(ERROR, "invalid range bound datum");
+			bound->datums[i] = val->constvalue;
+		}
+
+		i++;
+	}
+
+	return bound;
+}
+
+/*
+ * partition_rbound_cmp
+ *
+ * Return for two range bounds whether the 1st one (specified in datums1,
+ * kind1, and lower1) is <, =, or > the bound specified in *b2.
+ *
+ * Note that if the values of the two range bounds compare equal, then we take
+ * into account whether they are upper or lower bounds, and an upper bound is
+ * considered to be smaller than a lower bound. This is important to the way
+ * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
+ * structure, which only stores the upper bound of a common boundary between
+ * two contiguous partitions.
+ */
+static int32
+partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2)
+{
+	int32		cmpval = 0;		/* placate compiler */
+	int			i;
+	Datum	   *datums2 = b2->datums;
+	PartitionRangeDatumKind *kind2 = b2->kind;
+	bool		lower2 = b2->lower;
+
+	for (i = 0; i < key->partnatts; i++)
+	{
+		/*
+		 * First, handle cases where the column is unbounded, which should not
+		 * invoke the comparison procedure, and should not consider any later
+		 * columns. Note that the PartitionRangeDatumKind enum elements
+		 * compare the same way as the values they represent.
+		 */
+		if (kind1[i] < kind2[i])
+			return -1;
+		else if (kind1[i] > kind2[i])
+			return 1;
+		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+
+			/*
+			 * The column bounds are both MINVALUE or both MAXVALUE. No later
+			 * columns should be considered, but we still need to compare
+			 * whether they are upper or lower bounds.
+			 */
+			break;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 datums1[i],
+												 datums2[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	/*
+	 * If the comparison is anything other than equal, we're done. If they
+	 * compare equal though, we still have to consider whether the boundaries
+	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
+	 * two.
+	 */
+	if (cmpval == 0 && lower1 != lower2)
+		cmpval = lower1 ? 1 : -1;
+
+	return cmpval;
+}
+
+/* Used when sorting range bounds across all range partitions */
+static int32
+qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
+{
+	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
+	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
+	PartitionKey key = (PartitionKey) arg;
+
+	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+}
+
+/*
+ * partition_list_bsearch
+ *		Returns the index of the greatest bound datum that is less than equal
+ * 		to the given value or -1 if all of the bound datums are greater
+ *
+ * *is_equal is set to true if the bound datum at the returned index is equal
+ * to the input value.
+ */
+static int
+partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+												 key->partcollation[0],
+												 boundinfo->datums[mid][0],
+												 value));
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_rbound_datum_cmp
+ *
+ * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
+ * is <, =, or > partition key of tuple (tuple_datums)
+ */
+static int32
+partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums)
+{
+	int			i;
+	int32		cmpval = -1;
+
+	for (i = 0; i < n_tuple_datums; i++)
+	{
+		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
+			return -1;
+		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
+			return 1;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 rb_datums[i],
+												 tuple_datums[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	return cmpval;
+}
+
+/*
+ * partition_range_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given range bound or -1 if all of the range bounds are
+ *		greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input range bound
+ */
+static int
+partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_cmp(key,
+									  boundinfo->datums[mid],
+									  boundinfo->kind[mid],
+									  (boundinfo->indexes[mid] == -1),
+									  probe);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_range_datum_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given tuple or -1 if all of the range bounds are greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input tuple.
+ */
+static int
+partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_datum_cmp(key,
+											boundinfo->datums[mid],
+											boundinfo->kind[mid],
+											values,
+											nvalues);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_hash_bsearch
+ *		Returns the index of the greatest (modulus, remainder) pair that is
+ *		less than or equal to the given (modulus, remainder) pair or -1 if
+ *		all of them are greater
+ */
+static int
+partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval,
+					bound_modulus,
+					bound_remainder;
+
+		mid = (lo + hi + 1) / 2;
+		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
+		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
+		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
+									  modulus, remainder);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+
+			if (cmpval == 0)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * get_partition_bound_num_indexes
+ *
+ * Returns the number of the entries in the partition bound indexes array.
+ */
+static int
+get_partition_bound_num_indexes(PartitionBoundInfo bound)
+{
+	int			num_indexes;
+
+	Assert(bound);
+
+	switch (bound->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * The number of the entries in the indexes array is same as the
+			 * greatest modulus.
+			 */
+			num_indexes = get_greatest_modulus(bound);
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			num_indexes = bound->ndatums;
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			/* Range partitioned table has an extra index. */
+			num_indexes = bound->ndatums + 1;
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) bound->strategy);
+	}
+
+	return num_indexes;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 1ebf9c4ed2..32fba90be7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -42,7 +42,6 @@
 #include "catalog/index.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_amproc.h"
 #include "catalog/pg_attrdef.h"
@@ -81,6 +80,7 @@
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partcache.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -261,7 +261,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
 static Relation AllocateRelationDesc(Form_pg_class relp);
 static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
 static void RelationBuildTupleDesc(Relation relation);
-static void RelationBuildPartitionKey(Relation relation);
 static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
 static void RelationInitPhysicalAddr(Relation relation);
 static void load_critical_index(Oid indexoid, Oid heapoid);
@@ -809,209 +808,6 @@ RelationBuildRuleLock(Relation relation)
 }
 
 /*
- * RelationBuildPartitionKey
- *		Build and attach to relcache partition key data of relation
- *
- * Partitioning key data is a complex structure; to avoid complicated logic to
- * free individual elements whenever the relcache entry is flushed, we give it
- * its own memory context, child of CacheMemoryContext, which can easily be
- * deleted on its own.  To avoid leaking memory in that context in case of an
- * error partway through this function, the context is initially created as a
- * child of CurTransactionContext and only re-parented to CacheMemoryContext
- * at the end, when no further errors are possible.  Also, we don't make this
- * context the current context except in very brief code sections, out of fear
- * that some of our callees allocate memory on their own which would be leaked
- * permanently.
- */
-static void
-RelationBuildPartitionKey(Relation relation)
-{
-	Form_pg_partitioned_table form;
-	HeapTuple	tuple;
-	bool		isnull;
-	int			i;
-	PartitionKey key;
-	AttrNumber *attrs;
-	oidvector  *opclass;
-	oidvector  *collation;
-	ListCell   *partexprs_item;
-	Datum		datum;
-	MemoryContext partkeycxt,
-				oldcxt;
-	int16		procnum;
-
-	tuple = SearchSysCache1(PARTRELID,
-							ObjectIdGetDatum(RelationGetRelid(relation)));
-
-	/*
-	 * The following happens when we have created our pg_class entry but not
-	 * the pg_partitioned_table entry yet.
-	 */
-	if (!HeapTupleIsValid(tuple))
-		return;
-
-	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
-											   RelationGetRelationName(relation),
-											   MEMCONTEXT_COPY_NAME,
-											   ALLOCSET_SMALL_SIZES);
-
-	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
-												sizeof(PartitionKeyData));
-
-	/* Fixed-length attributes */
-	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	key->strategy = form->partstrat;
-	key->partnatts = form->partnatts;
-
-	/*
-	 * We can rely on the first variable-length attribute being mapped to the
-	 * relevant field of the catalog's C struct, because all previous
-	 * attributes are non-nullable and fixed-length.
-	 */
-	attrs = form->partattrs.values;
-
-	/* But use the hard way to retrieve further variable-length attributes */
-	/* Operator class */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partclass, &isnull);
-	Assert(!isnull);
-	opclass = (oidvector *) DatumGetPointer(datum);
-
-	/* Collation */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partcollation, &isnull);
-	Assert(!isnull);
-	collation = (oidvector *) DatumGetPointer(datum);
-
-	/* Expressions */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partexprs, &isnull);
-	if (!isnull)
-	{
-		char	   *exprString;
-		Node	   *expr;
-
-		exprString = TextDatumGetCString(datum);
-		expr = stringToNode(exprString);
-		pfree(exprString);
-
-		/*
-		 * Run the expressions through const-simplification since the planner
-		 * will be comparing them to similarly-processed qual clause operands,
-		 * and may fail to detect valid matches without this step; fix
-		 * opfuncids while at it.  We don't need to bother with
-		 * canonicalize_qual() though, because partition expressions are not
-		 * full-fledged qualification clauses.
-		 */
-		expr = eval_const_expressions(NULL, expr);
-		fix_opfuncids(expr);
-
-		oldcxt = MemoryContextSwitchTo(partkeycxt);
-		key->partexprs = (List *) copyObject(expr);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
-	oldcxt = MemoryContextSwitchTo(partkeycxt);
-	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
-	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
-
-	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-
-	/* Gather type and collation info as well */
-	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
-	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
-	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
-	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
-	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	MemoryContextSwitchTo(oldcxt);
-
-	/* determine support function number to search for */
-	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
-		HASHEXTENDED_PROC : BTORDER_PROC;
-
-	/* Copy partattrs and fill other per-attribute info */
-	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		AttrNumber	attno = key->partattrs[i];
-		HeapTuple	opclasstup;
-		Form_pg_opclass opclassform;
-		Oid			funcid;
-
-		/* Collect opfamily information */
-		opclasstup = SearchSysCache1(CLAOID,
-									 ObjectIdGetDatum(opclass->values[i]));
-		if (!HeapTupleIsValid(opclasstup))
-			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
-
-		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
-		key->partopfamily[i] = opclassform->opcfamily;
-		key->partopcintype[i] = opclassform->opcintype;
-
-		/* Get a support function for the specified opfamily and datatypes */
-		funcid = get_opfamily_proc(opclassform->opcfamily,
-								   opclassform->opcintype,
-								   opclassform->opcintype,
-								   procnum);
-		if (!OidIsValid(funcid))
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
-							NameStr(opclassform->opcname),
-							(key->strategy == PARTITION_STRATEGY_HASH) ?
-							"hash" : "btree",
-							procnum,
-							format_type_be(opclassform->opcintype))));
-
-		fmgr_info(funcid, &key->partsupfunc[i]);
-
-		/* Collation */
-		key->partcollation[i] = collation->values[i];
-
-		/* Collect type information */
-		if (attno != 0)
-		{
-			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
-
-			key->parttypid[i] = att->atttypid;
-			key->parttypmod[i] = att->atttypmod;
-			key->parttypcoll[i] = att->attcollation;
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			key->parttypid[i] = exprType(lfirst(partexprs_item));
-			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
-			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
-
-			partexprs_item = lnext(partexprs_item);
-		}
-		get_typlenbyvalalign(key->parttypid[i],
-							 &key->parttyplen[i],
-							 &key->parttypbyval[i],
-							 &key->parttypalign[i]);
-
-		ReleaseSysCache(opclasstup);
-	}
-
-	ReleaseSysCache(tuple);
-
-	/*
-	 * Success --- reparent our context and make the relcache point to the
-	 * newly constructed key
-	 */
-	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
-	relation->rd_partkeycxt = partkeycxt;
-	relation->rd_partkey = key;
-}
-
-/*
  *		equalRuleLocks
  *
  *		Determine whether two RuleLocks are equivalent
diff --git a/src/backend/utils/misc/pg_controldata.c b/src/backend/utils/misc/pg_controldata.c
index 8ab7d1337f..87b749a9c4 100644
--- a/src/backend/utils/misc/pg_controldata.c
+++ b/src/backend/utils/misc/pg_controldata.c
@@ -15,6 +15,7 @@
 
 #include "postgres.h"
 
+#include "access/xlog.h"
 #include "funcapi.h"
 #include "miscadmin.h"
 #include "access/htup_details.h"
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2faf0ca26e..b2bdd05005 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -14,63 +14,19 @@
 #define PARTITION_H
 
 #include "fmgr.h"
-#include "executor/tuptable.h"
-#include "nodes/execnodes.h"
-#include "parser/parse_node.h"
 #include "utils/rel.h"
 
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-/*
- * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
- * associated with partitioned tables as part of its partition descriptor.
- *
- * The internal structure is opaque outside partition.c.
- */
-typedef struct PartitionBoundInfoData *PartitionBoundInfo;
-
-/*
- * Information about partitions of a partitioned table.
- */
-typedef struct PartitionDescData
-{
-	int			nparts;			/* Number of partitions */
-	Oid		   *oids;			/* OIDs of partitions */
-	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
-} PartitionDescData;
-
-typedef struct PartitionDescData *PartitionDesc;
-
-extern void RelationBuildPartitionDesc(Relation relation);
-extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
-					   bool *parttypbyval, PartitionBoundInfo b1,
-					   PartitionBoundInfo b2);
-extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key);
-
-extern void check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec);
 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 fromrel_varno,
-						Relation to_rel, Relation from_rel,
-						bool *found_whole_row);
-extern List *RelationGetPartitionQual(Relation rel);
-extern Expr *get_partition_qual_relid(Oid relid);
-extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr);
 
-extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
 extern Oid	get_default_partition_oid(Oid parentId);
 extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
-extern int get_partition_for_tuple(Relation relation, Datum *values,
-						bool *isnull);
-
 #endif							/* PARTITION_H */
diff --git a/src/include/commands/tablecmds.h b/src/include/commands/tablecmds.h
index 06e5180a30..e786df6f81 100644
--- a/src/include/commands/tablecmds.h
+++ b/src/include/commands/tablecmds.h
@@ -18,7 +18,6 @@
 #include "catalog/dependency.h"
 #include "catalog/objectaddress.h"
 #include "nodes/parsenodes.h"
-#include "catalog/partition.h"
 #include "storage/lock.h"
 #include "utils/relcache.h"
 
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 3df9c498bb..c53dfcc265 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -13,10 +13,10 @@
 #ifndef EXECPARTITION_H
 #define EXECPARTITION_H
 
-#include "catalog/partition.h"
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "utils/partcache.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 1d824eff36..75914028f5 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -14,7 +14,6 @@
 #ifndef EXECUTOR_H
 #define EXECUTOR_H
 
-#include "catalog/partition.h"
 #include "executor/execdesc.h"
 #include "nodes/parsenodes.h"
 #include "utils/memutils.h"
diff --git a/src/include/rewrite/rewriteManip.h b/src/include/rewrite/rewriteManip.h
index f0299bc703..7ca8a81931 100644
--- a/src/include/rewrite/rewriteManip.h
+++ b/src/include/rewrite/rewriteManip.h
@@ -15,6 +15,7 @@
 #define REWRITEMANIP_H
 
 #include "nodes/parsenodes.h"
+#include "utils/rel.h"
 
 
 typedef struct replace_rte_variables_context replace_rte_variables_context;
@@ -73,6 +74,9 @@ extern Node *map_variable_attnos(Node *node,
 					int target_varno, int sublevels_up,
 					const AttrNumber *attno_map, int map_length,
 					Oid to_rowtype, bool *found_whole_row);
+extern List *map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel,
+						bool *found_whole_row);
 
 extern Node *ReplaceVarsFromTargetList(Node *node,
 						  int target_varno, int sublevels_up,
diff --git a/src/include/utils/partcache.h b/src/include/utils/partcache.h
new file mode 100644
index 0000000000..5d4caeda3a
--- /dev/null
+++ b/src/include/utils/partcache.h
@@ -0,0 +1,191 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.h
+ *		Header file for partitioning related cached data structures and
+ *		manipulation functions
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partcache.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTCACHE_H
+#define PARTCACHE_H
+
+#include "postgres.h"
+
+#include "fmgr.h"
+#include "nodes/parsenodes.h"
+#include "utils/lsyscache.h"
+#include "utils/relcache.h"
+
+/*
+ * Information about the partition key of a relation
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partitioning strategy */
+	int16		partnatts;		/* number of columns in the partition key */
+	AttrNumber *partattrs;		/* attribute numbers of columns in the
+								 * partition key */
+	List	   *partexprs;		/* list of expressions in the partitioning
+								 * key, or NIL */
+
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+
+	/* Partitioning collation per attribute */
+	Oid		   *partcollation;
+
+	/* Type information per attribute */
+	Oid		   *parttypid;
+	int32	   *parttypmod;
+	int16	   *parttyplen;
+	bool	   *parttypbyval;
+	char	   *parttypalign;
+	Oid		   *parttypcoll;
+}			PartitionKeyData;
+
+typedef struct PartitionKeyData *PartitionKey;
+
+typedef struct PartitionBoundInfoData *PartitionBoundInfo;
+
+/*
+ * Information about partitions of a partitioned table.
+ */
+typedef struct PartitionDescData
+{
+	int			nparts;			/* Number of partitions */
+	Oid		   *oids;			/* OIDs of partitions */
+	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
+} PartitionDescData;
+
+typedef struct PartitionDescData *PartitionDesc;
+
+/*
+ * Information about bounds of a partitioned relation
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+	char		strategy;		/* hash, list or range? */
+	int			ndatums;		/* Length of the datums following array */
+	Datum	  **datums;
+	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+									 * NULL for hash and list partitioned
+									 * tables */
+	int		   *indexes;		/* Partition indexes */
+	int			null_index;		/* Index of the null-accepting partition; -1
+								 * if there isn't one */
+	int			default_index;	/* Index of the default partition; -1 if there
+								 * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * PartitionKey inquiry functions
+ */
+static inline int
+get_partition_strategy(PartitionKey key)
+{
+	return key->strategy;
+}
+
+static inline int
+get_partition_natts(PartitionKey key)
+{
+	return key->partnatts;
+}
+
+static inline List *
+get_partition_exprs(PartitionKey key)
+{
+	return key->partexprs;
+}
+
+/*
+ * PartitionKey inquiry functions - one column
+ */
+static inline int16
+get_partition_col_attnum(PartitionKey key, int col)
+{
+	return key->partattrs[col];
+}
+
+static inline Oid
+get_partition_col_typid(PartitionKey key, int col)
+{
+	return key->parttypid[col];
+}
+
+static inline int32
+get_partition_col_typmod(PartitionKey key, int col)
+{
+	return key->parttypmod[col];
+}
+
+extern void RelationBuildPartitionKey(Relation relation);
+extern void RelationBuildPartitionDesc(Relation relation);
+extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
+					   bool *parttypbyval, PartitionBoundInfo b1,
+					   PartitionBoundInfo b2);
+
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
+
+extern void check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec);
+
+extern List *RelationGetPartitionQual(Relation rel);
+extern Expr *get_partition_qual_relid(Oid relid);
+
+extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr);
+
+extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+
+extern int get_greatest_modulus(PartitionBoundInfo b);
+extern uint64 compute_hash_value(PartitionKey key, Datum *values,
+						bool *isnull);
+
+/* For tuple routing */
+extern int get_partition_for_tuple(Relation relation, Datum *values,
+						bool *isnull);
+
+#endif							/* PARTCACHE_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index aa8add544a..b531ef0121 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -24,6 +24,7 @@
 #include "rewrite/prs2lock.h"
 #include "storage/block.h"
 #include "storage/relfilenode.h"
+#include "utils/partcache.h"
 #include "utils/relcache.h"
 #include "utils/reltrigger.h"
 
@@ -47,36 +48,6 @@ typedef struct LockInfoData
 typedef LockInfoData *LockInfo;
 
 /*
- * Information about the partition key of a relation
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partitioning strategy */
-	int16		partnatts;		/* number of columns in the partition key */
-	AttrNumber *partattrs;		/* attribute numbers of columns in the
-								 * partition key */
-	List	   *partexprs;		/* list of expressions in the partitioning
-								 * key, or NIL */
-
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-
-	/* Partitioning collation per attribute */
-	Oid		   *partcollation;
-
-	/* Type information per attribute */
-	Oid		   *parttypid;
-	int32	   *parttypmod;
-	int16	   *parttyplen;
-	bool	   *parttypbyval;
-	char	   *parttypalign;
-	Oid		   *parttypcoll;
-}			PartitionKeyData;
-
-typedef struct PartitionKeyData *PartitionKey;
-
-/*
  * Here are the contents of a relation cache entry.
  */
 
@@ -593,48 +564,6 @@ typedef struct ViewOptions
 #define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
 
 /*
- * PartitionKey inquiry functions
- */
-static inline int
-get_partition_strategy(PartitionKey key)
-{
-	return key->strategy;
-}
-
-static inline int
-get_partition_natts(PartitionKey key)
-{
-	return key->partnatts;
-}
-
-static inline List *
-get_partition_exprs(PartitionKey key)
-{
-	return key->partexprs;
-}
-
-/*
- * PartitionKey inquiry functions - one column
- */
-static inline int16
-get_partition_col_attnum(PartitionKey key, int col)
-{
-	return key->partattrs[col];
-}
-
-static inline Oid
-get_partition_col_typid(PartitionKey key, int col)
-{
-	return key->parttypid[col];
-}
-
-static inline int32
-get_partition_col_typmod(PartitionKey key, int col)
-{
-	return key->parttypmod[col];
-}
-
-/*
  * RelationGetPartitionDesc
  *		Returns partition descriptor for a relation.
  */
-- 
2.11.0

#4Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#3)
Re: reorganizing partitioning code (was: Re: [HACKERS] path toward faster partition pruning)

This is looking attractive.

Please don't #include postgres.h in partcache.h. That's per policy.

Why do you need to #include parsenodes.h in partcache.h?

I think rewriteManip.h can do with just relcache.h rather than rel.h
(probably partition.h can do likewise)

thanks

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#5Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#3)
Re: reorganizing partitioning code (was: Re: [HACKERS] path toward faster partition pruning)

BTW may I suggest that

git config diff.algorithm=histogram

results in better diffs?

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#6Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Alvaro Herrera (#5)
Re: reorganizing partitioning code

On 2018/02/15 5:30, Alvaro Herrera wrote:

BTW may I suggest that

git config diff.algorithm=histogram

results in better diffs?

Aha! That's much better.

Thanks,
Amit

#7Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Alvaro Herrera (#4)
1 attachment(s)
Re: reorganizing partitioning code

Thanks for the review.

On 2018/02/15 5:25, Alvaro Herrera wrote:

This is looking attractive.

Please don't #include postgres.h in partcache.h. That's per policy.

Oops, fixed.

Why do you need to #include parsenodes.h in partcache.h?

I thought it was needed because there was this:

extern void check_new_partition_bound(char *relname, Relation parent,
PartitionBoundSpec *spec);

in partcache.h and PartitionBoundSpec is defined in parsenodes.h.
Removing the #include turned out to be fine, that is, after I put it in
partition.h instead.

I think rewriteManip.h can do with just relcache.h rather than rel.h
(probably partition.h can do likewise)

Hmm. map_partition_varattnos() that I moved to rewriteManip.c wants to
use RelationGetDescr(), but partition.h was fine.

Attached updated patch.

Thanks,
Amit

Attachments:

v3-0001-Reorganize-partitioning-code.patchtext/plain; charset=UTF-8; name=v3-0001-Reorganize-partitioning-code.patchDownload
From 20dd2534f727f2fbd963ed1001091f9bd324824e Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Tue, 13 Feb 2018 15:59:30 +0900
Subject: [PATCH v3] Reorganize partitioning code

---
 src/backend/catalog/partition.c         | 1979 +----------------------------
 src/backend/executor/execExpr.c         |    1 +
 src/backend/executor/execMain.c         |    1 -
 src/backend/executor/execPartition.c    |    1 +
 src/backend/executor/nodeModifyTable.c  |    1 +
 src/backend/libpq/hba.c                 |    1 +
 src/backend/optimizer/path/joinrels.c   |    2 +-
 src/backend/optimizer/prep/prepunion.c  |    2 +-
 src/backend/optimizer/util/plancat.c    |    1 -
 src/backend/optimizer/util/relnode.c    |    2 +-
 src/backend/rewrite/rewriteManip.c      |   45 +
 src/backend/tcop/utility.c              |    1 +
 src/backend/utils/adt/ruleutils.c       |    1 -
 src/backend/utils/cache/Makefile        |    6 +-
 src/backend/utils/cache/partcache.c     | 2118 +++++++++++++++++++++++++++++++
 src/backend/utils/cache/relcache.c      |  206 +--
 src/backend/utils/misc/pg_controldata.c |    1 +
 src/include/catalog/partition.h         |   47 +-
 src/include/commands/tablecmds.h        |    1 -
 src/include/executor/execPartition.h    |    2 +-
 src/include/executor/executor.h         |    1 -
 src/include/rewrite/rewriteManip.h      |    4 +
 src/include/utils/partcache.h           |  188 +++
 src/include/utils/rel.h                 |   73 +-
 24 files changed, 2373 insertions(+), 2312 deletions(-)
 create mode 100644 src/backend/utils/cache/partcache.c
 create mode 100644 src/include/utils/partcache.h

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 31c80c7f1a..d0f1c686ed 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,11 +15,7 @@
 
 #include "postgres.h"
 
-#include "access/hash.h"
-#include "access/heapam.h"
 #include "access/htup_details.h"
-#include "access/nbtree.h"
-#include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
@@ -35,7 +31,6 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
-#include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
 #include "optimizer/prep.h"
@@ -48,102 +43,12 @@
 #include "utils/datum.h"
 #include "utils/fmgroids.h"
 #include "utils/hashutils.h"
-#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
-#include "utils/ruleutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-	char		strategy;		/* hash, list or range? */
-	int			ndatums;		/* Length of the datums following array */
-	Datum	  **datums;
-	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-									 * NULL for hash and list partitioned
-									 * tables */
-	int		   *indexes;		/* Partition indexes */
-	int			null_index;		/* Index of the null-accepting partition; -1
-								 * if there isn't one */
-	int			default_index;	/* Index of the default partition; -1 if there
-								 * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-	int			modulus;
-	int			remainder;
-	int			index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-	int			index;
-	Datum		value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-	int			index;
-	Datum	   *datums;			/* range bound datums */
-	PartitionRangeDatumKind *kind;	/* the kind of each datum */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
-static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
-static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
-							   void *arg);
-static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
-						   void *arg);
-
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
@@ -159,1069 +64,11 @@ static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 				   bool for_default);
 static List *get_range_nulltest(PartitionKey key);
-static List *generate_partition_qual(Relation rel);
-
-static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
-					 List *datums, bool lower);
-static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
-					 int remainder2);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal);
-static int partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder);
-
-static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int	get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 
 /* SQL-callable function for use in hash partition CHECK constraints */
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 /*
- * RelationBuildPartitionDesc
- *		Form rel's partition descriptor
- *
- * Not flushed from the cache by RelationClearRelation() unless changed because
- * of addition or removal of partition.
- */
-void
-RelationBuildPartitionDesc(Relation rel)
-{
-	List	   *inhoids,
-			   *partoids;
-	Oid		   *oids = NULL;
-	List	   *boundspecs = NIL;
-	ListCell   *cell;
-	int			i,
-				nparts;
-	PartitionKey key = RelationGetPartitionKey(rel);
-	PartitionDesc result;
-	MemoryContext oldcxt;
-
-	int			ndatums = 0;
-	int			default_index = -1;
-
-	/* Hash partitioning specific */
-	PartitionHashBound **hbounds = NULL;
-
-	/* List partitioning specific */
-	PartitionListValue **all_values = NULL;
-	int			null_index = -1;
-
-	/* Range partitioning specific */
-	PartitionRangeBound **rbounds = NULL;
-
-	/*
-	 * The following could happen in situations where rel has a pg_class entry
-	 * but not the pg_partitioned_table entry yet.
-	 */
-	if (key == NULL)
-		return;
-
-	/* Get partition oids from pg_inherits */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
-
-	/* Collect bound spec nodes in a list */
-	i = 0;
-	partoids = NIL;
-	foreach(cell, inhoids)
-	{
-		Oid			inhrelid = lfirst_oid(cell);
-		HeapTuple	tuple;
-		Datum		datum;
-		bool		isnull;
-		Node	   *boundspec;
-
-		tuple = SearchSysCache1(RELOID, inhrelid);
-		if (!HeapTupleIsValid(tuple))
-			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
-
-		/*
-		 * It is possible that the pg_class tuple of a partition has not been
-		 * updated yet to set its relpartbound field.  The only case where
-		 * this happens is when we open the parent relation to check using its
-		 * partition descriptor that a new partition's bound does not overlap
-		 * some existing partition.
-		 */
-		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
-		{
-			ReleaseSysCache(tuple);
-			continue;
-		}
-
-		datum = SysCacheGetAttr(RELOID, tuple,
-								Anum_pg_class_relpartbound,
-								&isnull);
-		Assert(!isnull);
-		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
-
-		/*
-		 * Sanity check: If the PartitionBoundSpec says this is the default
-		 * partition, its OID should correspond to whatever's stored in
-		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
-		 */
-		if (castNode(PartitionBoundSpec, boundspec)->is_default)
-		{
-			Oid			partdefid;
-
-			partdefid = get_default_partition_oid(RelationGetRelid(rel));
-			if (partdefid != inhrelid)
-				elog(ERROR, "expected partdefid %u, but got %u",
-					 inhrelid, partdefid);
-		}
-
-		boundspecs = lappend(boundspecs, boundspec);
-		partoids = lappend_oid(partoids, inhrelid);
-		ReleaseSysCache(tuple);
-	}
-
-	nparts = list_length(partoids);
-
-	if (nparts > 0)
-	{
-		oids = (Oid *) palloc(nparts * sizeof(Oid));
-		i = 0;
-		foreach(cell, partoids)
-			oids[i++] = lfirst_oid(cell);
-
-		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_HASH)
-		{
-			ndatums = nparts;
-			hbounds = (PartitionHashBound **)
-				palloc(nparts * sizeof(PartitionHashBound *));
-
-			i = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-
-				if (spec->strategy != PARTITION_STRATEGY_HASH)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				hbounds[i] = (PartitionHashBound *)
-					palloc(sizeof(PartitionHashBound));
-
-				hbounds[i]->modulus = spec->modulus;
-				hbounds[i]->remainder = spec->remainder;
-				hbounds[i]->index = i;
-				i++;
-			}
-
-			/* Sort all the bounds in ascending order */
-			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
-				  qsort_partition_hbound_cmp);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_LIST)
-		{
-			List	   *non_null_values = NIL;
-
-			/*
-			 * Create a unified list of non-null values across all partitions.
-			 */
-			i = 0;
-			null_index = -1;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				ListCell   *c;
-
-				if (spec->strategy != PARTITION_STRATEGY_LIST)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the list of non-null
-				 * datums for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i;
-					i++;
-					continue;
-				}
-
-				foreach(c, spec->listdatums)
-				{
-					Const	   *val = castNode(Const, lfirst(c));
-					PartitionListValue *list_value = NULL;
-
-					if (!val->constisnull)
-					{
-						list_value = (PartitionListValue *)
-							palloc0(sizeof(PartitionListValue));
-						list_value->index = i;
-						list_value->value = val->constvalue;
-					}
-					else
-					{
-						/*
-						 * Never put a null into the values array, flag
-						 * instead for the code further down below where we
-						 * construct the actual relcache struct.
-						 */
-						if (null_index != -1)
-							elog(ERROR, "found null more than once");
-						null_index = i;
-					}
-
-					if (list_value)
-						non_null_values = lappend(non_null_values,
-												  list_value);
-				}
-
-				i++;
-			}
-
-			ndatums = list_length(non_null_values);
-
-			/*
-			 * Collect all list values in one array. Alongside the value, we
-			 * also save the index of partition the value comes from.
-			 */
-			all_values = (PartitionListValue **) palloc(ndatums *
-														sizeof(PartitionListValue *));
-			i = 0;
-			foreach(cell, non_null_values)
-			{
-				PartitionListValue *src = lfirst(cell);
-
-				all_values[i] = (PartitionListValue *)
-					palloc(sizeof(PartitionListValue));
-				all_values[i]->value = src->value;
-				all_values[i]->index = src->index;
-				i++;
-			}
-
-			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
-					  qsort_partition_list_value_cmp, (void *) key);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_RANGE)
-		{
-			int			k;
-			PartitionRangeBound **all_bounds,
-					   *prev;
-
-			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
-														  sizeof(PartitionRangeBound *));
-
-			/*
-			 * Create a unified list of range bounds across all the
-			 * partitions.
-			 */
-			i = ndatums = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				PartitionRangeBound *lower,
-						   *upper;
-
-				if (spec->strategy != PARTITION_STRATEGY_RANGE)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the allbounds array
-				 * for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i++;
-					continue;
-				}
-
-				lower = make_one_range_bound(key, i, spec->lowerdatums,
-											 true);
-				upper = make_one_range_bound(key, i, spec->upperdatums,
-											 false);
-				all_bounds[ndatums++] = lower;
-				all_bounds[ndatums++] = upper;
-				i++;
-			}
-
-			Assert(ndatums == nparts * 2 ||
-				   (default_index != -1 && ndatums == (nparts - 1) * 2));
-
-			/* Sort all the bounds in ascending order */
-			qsort_arg(all_bounds, ndatums,
-					  sizeof(PartitionRangeBound *),
-					  qsort_partition_rbound_cmp,
-					  (void *) key);
-
-			/* Save distinct bounds from all_bounds into rbounds. */
-			rbounds = (PartitionRangeBound **)
-				palloc(ndatums * sizeof(PartitionRangeBound *));
-			k = 0;
-			prev = NULL;
-			for (i = 0; i < ndatums; i++)
-			{
-				PartitionRangeBound *cur = all_bounds[i];
-				bool		is_distinct = false;
-				int			j;
-
-				/* Is the current bound distinct from the previous one? */
-				for (j = 0; j < key->partnatts; j++)
-				{
-					Datum		cmpval;
-
-					if (prev == NULL || cur->kind[j] != prev->kind[j])
-					{
-						is_distinct = true;
-						break;
-					}
-
-					/*
-					 * If the bounds are both MINVALUE or MAXVALUE, stop now
-					 * and treat them as equal, since any values after this
-					 * point must be ignored.
-					 */
-					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
-						break;
-
-					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
-											   key->partcollation[j],
-											   cur->datums[j],
-											   prev->datums[j]);
-					if (DatumGetInt32(cmpval) != 0)
-					{
-						is_distinct = true;
-						break;
-					}
-				}
-
-				/*
-				 * Only if the bound is distinct save it into a temporary
-				 * array i.e. rbounds which is later copied into boundinfo
-				 * datums array.
-				 */
-				if (is_distinct)
-					rbounds[k++] = all_bounds[i];
-
-				prev = cur;
-			}
-
-			/* Update ndatums to hold the count of distinct datums. */
-			ndatums = k;
-		}
-		else
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/* Now build the actual relcache partition descriptor */
-	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
-												  RelationGetRelationName(rel),
-												  MEMCONTEXT_COPY_NAME,
-												  ALLOCSET_DEFAULT_SIZES);
-	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
-
-	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
-	result->nparts = nparts;
-	if (nparts > 0)
-	{
-		PartitionBoundInfo boundinfo;
-		int		   *mapping;
-		int			next_index = 0;
-
-		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
-
-		boundinfo = (PartitionBoundInfoData *)
-			palloc0(sizeof(PartitionBoundInfoData));
-		boundinfo->strategy = key->strategy;
-		boundinfo->default_index = -1;
-		boundinfo->ndatums = ndatums;
-		boundinfo->null_index = -1;
-		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
-
-		/* Initialize mapping array with invalid values */
-		mapping = (int *) palloc(sizeof(int) * nparts);
-		for (i = 0; i < nparts; i++)
-			mapping[i] = -1;
-
-		switch (key->strategy)
-		{
-			case PARTITION_STRATEGY_HASH:
-				{
-					/* Modulus are stored in ascending order */
-					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
-
-					boundinfo->indexes = (int *) palloc(greatest_modulus *
-														sizeof(int));
-
-					for (i = 0; i < greatest_modulus; i++)
-						boundinfo->indexes[i] = -1;
-
-					for (i = 0; i < nparts; i++)
-					{
-						int			modulus = hbounds[i]->modulus;
-						int			remainder = hbounds[i]->remainder;
-
-						boundinfo->datums[i] = (Datum *) palloc(2 *
-																sizeof(Datum));
-						boundinfo->datums[i][0] = Int32GetDatum(modulus);
-						boundinfo->datums[i][1] = Int32GetDatum(remainder);
-
-						while (remainder < greatest_modulus)
-						{
-							/* overlap? */
-							Assert(boundinfo->indexes[remainder] == -1);
-							boundinfo->indexes[remainder] = i;
-							remainder += modulus;
-						}
-
-						mapping[hbounds[i]->index] = i;
-						pfree(hbounds[i]);
-					}
-					pfree(hbounds);
-					break;
-				}
-
-			case PARTITION_STRATEGY_LIST:
-				{
-					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
-
-					/*
-					 * Copy values.  Indexes of individual values are mapped
-					 * to canonical values so that they match for any two list
-					 * partitioned tables with same number of partitions and
-					 * same lists per partition.  One way to canonicalize is
-					 * to assign the index in all_values[] of the smallest
-					 * value of each partition, as the index of all of the
-					 * partition's values.
-					 */
-					for (i = 0; i < ndatums; i++)
-					{
-						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
-						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
-															key->parttypbyval[0],
-															key->parttyplen[0]);
-
-						/* If the old index has no mapping, assign one */
-						if (mapping[all_values[i]->index] == -1)
-							mapping[all_values[i]->index] = next_index++;
-
-						boundinfo->indexes[i] = mapping[all_values[i]->index];
-					}
-
-					/*
-					 * If null-accepting partition has no mapped index yet,
-					 * assign one.  This could happen if such partition
-					 * accepts only null and hence not covered in the above
-					 * loop which only handled non-null values.
-					 */
-					if (null_index != -1)
-					{
-						Assert(null_index >= 0);
-						if (mapping[null_index] == -1)
-							mapping[null_index] = next_index++;
-						boundinfo->null_index = mapping[null_index];
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						/*
-						 * The default partition accepts any value not
-						 * specified in the lists of other partitions, hence
-						 * it should not get mapped index while assigning
-						 * those for non-null datums.
-						 */
-						Assert(default_index >= 0 &&
-							   mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-
-					/* All partition must now have a valid mapping */
-					Assert(next_index == nparts);
-					break;
-				}
-
-			case PARTITION_STRATEGY_RANGE:
-				{
-					boundinfo->kind = (PartitionRangeDatumKind **)
-						palloc(ndatums *
-							   sizeof(PartitionRangeDatumKind *));
-					boundinfo->indexes = (int *) palloc((ndatums + 1) *
-														sizeof(int));
-
-					for (i = 0; i < ndatums; i++)
-					{
-						int			j;
-
-						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
-																sizeof(Datum));
-						boundinfo->kind[i] = (PartitionRangeDatumKind *)
-							palloc(key->partnatts *
-								   sizeof(PartitionRangeDatumKind));
-						for (j = 0; j < key->partnatts; j++)
-						{
-							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
-								boundinfo->datums[i][j] =
-									datumCopy(rbounds[i]->datums[j],
-											  key->parttypbyval[j],
-											  key->parttyplen[j]);
-							boundinfo->kind[i][j] = rbounds[i]->kind[j];
-						}
-
-						/*
-						 * There is no mapping for invalid indexes.
-						 *
-						 * Any lower bounds in the rbounds array have invalid
-						 * indexes assigned, because the values between the
-						 * previous bound (if there is one) and this (lower)
-						 * bound are not part of the range of any existing
-						 * partition.
-						 */
-						if (rbounds[i]->lower)
-							boundinfo->indexes[i] = -1;
-						else
-						{
-							int			orig_index = rbounds[i]->index;
-
-							/* If the old index has no mapping, assign one */
-							if (mapping[orig_index] == -1)
-								mapping[orig_index] = next_index++;
-
-							boundinfo->indexes[i] = mapping[orig_index];
-						}
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						Assert(default_index >= 0 && mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-					boundinfo->indexes[i] = -1;
-					break;
-				}
-
-			default:
-				elog(ERROR, "unexpected partition strategy: %d",
-					 (int) key->strategy);
-		}
-
-		result->boundinfo = boundinfo;
-
-		/*
-		 * Now assign OIDs from the original array into mapped indexes of the
-		 * result array.  Order of OIDs in the former is defined by the
-		 * catalog scan that retrieved them, whereas that in the latter is
-		 * defined by canonicalized representation of the partition bounds.
-		 */
-		for (i = 0; i < nparts; i++)
-			result->oids[mapping[i]] = oids[i];
-		pfree(mapping);
-	}
-
-	MemoryContextSwitchTo(oldcxt);
-	rel->rd_partdesc = result;
-}
-
-/*
- * Are two partition bound collections logically equal?
- *
- * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
- * This is also useful when b1 and b2 are bound collections of two separate
- * relations, respectively, because PartitionBoundInfo is a canonical
- * representation of partition bounds.
- */
-bool
-partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
-					   PartitionBoundInfo b1, PartitionBoundInfo b2)
-{
-	int			i;
-
-	if (b1->strategy != b2->strategy)
-		return false;
-
-	if (b1->ndatums != b2->ndatums)
-		return false;
-
-	if (b1->null_index != b2->null_index)
-		return false;
-
-	if (b1->default_index != b2->default_index)
-		return false;
-
-	if (b1->strategy == PARTITION_STRATEGY_HASH)
-	{
-		int			greatest_modulus = get_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_greatest_modulus(b2))
-			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their modulus
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
-	}
-	else
-	{
-		for (i = 0; i < b1->ndatums; i++)
-		{
-			int			j;
-
-			for (j = 0; j < partnatts; j++)
-			{
-				/* For range partitions, the bounds might not be finite. */
-				if (b1->kind != NULL)
-				{
-					/* The different kinds of bound all differ from each other */
-					if (b1->kind[i][j] != b2->kind[i][j])
-						return false;
-
-					/*
-					 * Non-finite bounds are equal without further
-					 * examination.
-					 */
-					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
-						continue;
-				}
-
-				/*
-				 * Compare the actual values. Note that it would be both
-				 * incorrect and unsafe to invoke the comparison operator
-				 * derived from the partitioning specification here.  It would
-				 * be incorrect because we want the relcache entry to be
-				 * updated for ANY change to the partition bounds, not just
-				 * those that the partitioning operator thinks are
-				 * significant.  It would be unsafe because we might reach
-				 * this code in the context of an aborted transaction, and an
-				 * arbitrary partitioning operator might not be safe in that
-				 * context.  datumIsEqual() should be simple enough to be
-				 * safe.
-				 */
-				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-								  parttypbyval[j], parttyplen[j]))
-					return false;
-			}
-
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-		}
-
-		/* There are ndatums+1 indexes in case of range partitions */
-		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
-			b1->indexes[i] != b2->indexes[i])
-			return false;
-	}
-	return true;
-}
-
-/*
- * Return a copy of given PartitionBoundInfo structure. The data types of bounds
- * are described by given partition key specification.
- */
-extern PartitionBoundInfo
-partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key)
-{
-	PartitionBoundInfo dest;
-	int			i;
-	int			ndatums;
-	int			partnatts;
-	int			num_indexes;
-
-	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
-
-	dest->strategy = src->strategy;
-	ndatums = dest->ndatums = src->ndatums;
-	partnatts = key->partnatts;
-
-	num_indexes = get_partition_bound_num_indexes(src);
-
-	/* List partitioned tables have only a single partition key. */
-	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
-
-	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
-
-	if (src->kind != NULL)
-	{
-		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
-														 sizeof(PartitionRangeDatumKind *));
-		for (i = 0; i < ndatums; i++)
-		{
-			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
-															   sizeof(PartitionRangeDatumKind));
-
-			memcpy(dest->kind[i], src->kind[i],
-				   sizeof(PartitionRangeDatumKind) * key->partnatts);
-		}
-	}
-	else
-		dest->kind = NULL;
-
-	for (i = 0; i < ndatums; i++)
-	{
-		int			j;
-
-		/*
-		 * For a corresponding to hash partition, datums array will have two
-		 * elements - modulus and remainder.
-		 */
-		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
-		int			natts = hash_part ? 2 : partnatts;
-
-		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
-
-		for (j = 0; j < natts; j++)
-		{
-			bool		byval;
-			int			typlen;
-
-			if (hash_part)
-			{
-				typlen = sizeof(int32); /* Always int4 */
-				byval = true;	/* int4 is pass-by-value */
-			}
-			else
-			{
-				byval = key->parttypbyval[j];
-				typlen = key->parttyplen[j];
-			}
-
-			if (dest->kind == NULL ||
-				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
-				dest->datums[i][j] = datumCopy(src->datums[i][j],
-											   byval, typlen);
-		}
-	}
-
-	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
-	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
-
-	dest->null_index = src->null_index;
-	dest->default_index = src->default_index;
-
-	return dest;
-}
-
-/*
- * check_new_partition_bound
- *
- * Checks if the new partition's bound overlaps any of the existing partitions
- * of parent.  Also performs additional checks as necessary per strategy.
- */
-void
-check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	PartitionBoundInfo boundinfo = partdesc->boundinfo;
-	ParseState *pstate = make_parsestate(NULL);
-	int			with = -1;
-	bool		overlap = false;
-
-	if (spec->is_default)
-	{
-		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
-			return;
-
-		/* Default partition already exists, error out. */
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
-				 parser_errposition(pstate, spec->location)));
-	}
-
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					Datum	  **datums = boundinfo->datums;
-					int			ndatums = boundinfo->ndatums;
-					int			greatest_modulus;
-					int			remainder;
-					int			offset;
-					bool		valid_modulus = true;
-					int			prev_modulus,	/* Previous largest modulus */
-								next_modulus;	/* Next largest modulus */
-
-					/*
-					 * Check rule that every modulus must be a factor of the
-					 * next larger modulus.  For example, if you have a bunch
-					 * of partitions that all have modulus 5, you can add a
-					 * new partition with modulus 10 or a new partition with
-					 * modulus 15, but you cannot add both a partition with
-					 * modulus 10 and a partition with modulus 15, because 10
-					 * is not a factor of 15.
-					 *
-					 * Get the greatest (modulus, remainder) pair contained in
-					 * boundinfo->datums that is less than or equal to the
-					 * (spec->modulus, spec->remainder) pair.
-					 */
-					offset = partition_hash_bsearch(key, boundinfo,
-													spec->modulus,
-													spec->remainder);
-					if (offset < 0)
-					{
-						next_modulus = DatumGetInt32(datums[0][0]);
-						valid_modulus = (next_modulus % spec->modulus) == 0;
-					}
-					else
-					{
-						prev_modulus = DatumGetInt32(datums[offset][0]);
-						valid_modulus = (spec->modulus % prev_modulus) == 0;
-
-						if (valid_modulus && (offset + 1) < ndatums)
-						{
-							next_modulus = DatumGetInt32(datums[offset + 1][0]);
-							valid_modulus = (next_modulus % spec->modulus) == 0;
-						}
-					}
-
-					if (!valid_modulus)
-						ereport(ERROR,
-								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
-
-					greatest_modulus = get_greatest_modulus(boundinfo);
-					remainder = spec->remainder;
-
-					/*
-					 * Normally, the lowest remainder that could conflict with
-					 * the new partition is equal to the remainder specified
-					 * for the new partition, but when the new partition has a
-					 * modulus higher than any used so far, we need to adjust.
-					 */
-					if (remainder >= greatest_modulus)
-						remainder = remainder % greatest_modulus;
-
-					/* Check every potentially-conflicting remainder. */
-					do
-					{
-						if (boundinfo->indexes[remainder] != -1)
-						{
-							overlap = true;
-							with = boundinfo->indexes[remainder];
-							break;
-						}
-						remainder += spec->modulus;
-					} while (remainder < greatest_modulus);
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_LIST:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-
-				if (partdesc->nparts > 0)
-				{
-					ListCell   *cell;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo) ||
-							partition_bound_has_default(boundinfo)));
-
-					foreach(cell, spec->listdatums)
-					{
-						Const	   *val = castNode(Const, lfirst(cell));
-
-						if (!val->constisnull)
-						{
-							int			offset;
-							bool		equal;
-
-							offset = partition_list_bsearch(key, boundinfo,
-															val->constvalue,
-															&equal);
-							if (offset >= 0 && equal)
-							{
-								overlap = true;
-								with = boundinfo->indexes[offset];
-								break;
-							}
-						}
-						else if (partition_bound_accepts_nulls(boundinfo))
-						{
-							overlap = true;
-							with = boundinfo->null_index;
-							break;
-						}
-					}
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				PartitionRangeBound *lower,
-						   *upper;
-
-				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
-				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
-
-				/*
-				 * First check if the resulting range would be empty with
-				 * specified lower and upper bounds
-				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
-				{
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-							 errmsg("empty range bound specified for partition \"%s\"",
-									relname),
-							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
-									   get_range_partbound_string(spec->lowerdatums),
-									   get_range_partbound_string(spec->upperdatums)),
-							 parser_errposition(pstate, spec->location)));
-				}
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					int			offset;
-					bool		equal;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_has_default(boundinfo)));
-
-					/*
-					 * Test whether the new lower bound (which is treated
-					 * inclusively as part of the new partition) lies inside
-					 * an existing partition, or in a gap.
-					 *
-					 * If it's inside an existing partition, the bound at
-					 * offset + 1 will be the upper bound of that partition,
-					 * and its index will be >= 0.
-					 *
-					 * If it's in a gap, the bound at offset + 1 will be the
-					 * lower bound of the next partition, and its index will
-					 * be -1. This is also true if there is no next partition,
-					 * since the index array is initialised with an extra -1
-					 * at the end.
-					 */
-					offset = partition_range_bsearch(key, boundinfo, lower,
-													 &equal);
-
-					if (boundinfo->indexes[offset + 1] < 0)
-					{
-						/*
-						 * Check that the new partition will fit in the gap.
-						 * For it to fit, the new upper bound must be less
-						 * than or equal to the lower bound of the next
-						 * partition, if there is one.
-						 */
-						if (offset + 1 < boundinfo->ndatums)
-						{
-							int32		cmpval;
-							Datum 	   *datums;
-							PartitionRangeDatumKind *kind;
-							bool		is_lower;
-
-							datums = boundinfo->datums[offset + 1];
-							kind = boundinfo->kind[offset + 1];
-							is_lower = (boundinfo->indexes[offset + 1] == -1);
-
-							cmpval = partition_rbound_cmp(key, datums, kind,
-														  is_lower, upper);
-							if (cmpval < 0)
-							{
-								/*
-								 * The new partition overlaps with the
-								 * existing partition between offset + 1 and
-								 * offset + 2.
-								 */
-								overlap = true;
-								with = boundinfo->indexes[offset + 2];
-							}
-						}
-					}
-					else
-					{
-						/*
-						 * The new partition overlaps with the existing
-						 * partition between offset and offset + 1.
-						 */
-						overlap = true;
-						with = boundinfo->indexes[offset + 1];
-					}
-				}
-
-				break;
-			}
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	if (overlap)
-	{
-		Assert(with >= 0);
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" would overlap partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[with])),
-				 parser_errposition(pstate, spec->location)));
-	}
-}
-
-/*
  * check_default_allows_bound
  *
  * This function checks if there exists a row in the default partition that
@@ -1457,99 +304,6 @@ get_qual_from_partbound(Relation rel, Relation parent,
 	return my_qual;
 }
 
-/*
- * map_partition_varattnos - maps varattno of any Vars in expr from the
- * attno's of 'from_rel' to the attno's of 'to_rel' partition, each of which
- * may be either a leaf partition or a partitioned table, but both of which
- * must be from the same partitioning hierarchy.
- *
- * Even though all of the same column names must be present in all relations
- * in the hierarchy, and they must also have the same types, the attnos may
- * be different.
- *
- * If found_whole_row is not NULL, *found_whole_row returns whether a
- * whole-row variable was found in the input expression.
- *
- * Note: this will work on any node tree, so really the argument and result
- * should be declared "Node *".  But a substantial majority of the callers
- * are working on Lists, so it's less messy to do the casts internally.
- */
-List *
-map_partition_varattnos(List *expr, int fromrel_varno,
-						Relation to_rel, Relation from_rel,
-						bool *found_whole_row)
-{
-	bool		my_found_whole_row = false;
-
-	if (expr != NIL)
-	{
-		AttrNumber *part_attnos;
-
-		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,
-											fromrel_varno, 0,
-											part_attnos,
-											RelationGetDescr(from_rel)->natts,
-											RelationGetForm(to_rel)->reltype,
-											&my_found_whole_row);
-	}
-
-	if (found_whole_row)
-		*found_whole_row = my_found_whole_row;
-
-	return expr;
-}
-
-/*
- * RelationGetPartitionQual
- *
- * Returns a list of partition quals
- */
-List *
-RelationGetPartitionQual(Relation rel)
-{
-	/* Quick exit */
-	if (!rel->rd_rel->relispartition)
-		return NIL;
-
-	return generate_partition_qual(rel);
-}
-
-/*
- * get_partition_qual_relid
- *
- * Returns an expression tree describing the passed-in relation's partition
- * constraint. If there is no partition constraint returns NULL; this can
- * happen if the default partition is the only partition.
- */
-Expr *
-get_partition_qual_relid(Oid relid)
-{
-	Relation	rel = heap_open(relid, AccessShareLock);
-	Expr	   *result = NULL;
-	List	   *and_args;
-
-	/* Do the work only if this relation is a partition. */
-	if (rel->rd_rel->relispartition)
-	{
-		and_args = generate_partition_qual(rel);
-
-		if (and_args == NIL)
-			result = NULL;
-		else if (list_length(and_args) > 1)
-			result = makeBoolExpr(AND_EXPR, and_args, -1);
-		else
-			result = linitial(and_args);
-	}
-
-	/* Keep the lock. */
-	heap_close(rel, NoLock);
-
-	return result;
-}
-
 /* Module-local functions */
 
 /*
@@ -2466,644 +1220,6 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 }
 
 /*
- * generate_partition_qual
- *
- * Generate partition predicate from rel's partition bound expression. The
- * function returns a NIL list if there is no predicate.
- *
- * Result expression tree is stored CacheMemoryContext to ensure it survives
- * as long as the relcache entry. But we should be running in a less long-lived
- * working context. To avoid leaking cache memory if this routine fails partway
- * through, we build in working memory and then copy the completed structure
- * into cache memory.
- */
-static List *
-generate_partition_qual(Relation rel)
-{
-	HeapTuple	tuple;
-	MemoryContext oldcxt;
-	Datum		boundDatum;
-	bool		isnull;
-	PartitionBoundSpec *bound;
-	List	   *my_qual = NIL,
-			   *result = NIL;
-	Relation	parent;
-	bool		found_whole_row;
-
-	/* Guard against stack overflow due to overly deep partition tree */
-	check_stack_depth();
-
-	/* Quick copy */
-	if (rel->rd_partcheck != NIL)
-		return copyObject(rel->rd_partcheck);
-
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
-					   AccessShareLock);
-
-	/* Get pg_class.relpartbound */
-	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(rel));
-
-	boundDatum = SysCacheGetAttr(RELOID, tuple,
-								 Anum_pg_class_relpartbound,
-								 &isnull);
-	if (isnull)					/* should not happen */
-		elog(ERROR, "relation \"%s\" has relpartbound = null",
-			 RelationGetRelationName(rel));
-	bound = castNode(PartitionBoundSpec,
-					 stringToNode(TextDatumGetCString(boundDatum)));
-	ReleaseSysCache(tuple);
-
-	my_qual = get_qual_from_partbound(rel, parent, bound);
-
-	/* 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;
-
-	/*
-	 * Change Vars to have partition's attnos instead of the parent's. We do
-	 * this after we concatenate the parent's quals, because we want every Var
-	 * in it to bear this relation's attnos. It's safe to assume varno = 1
-	 * here.
-	 */
-	result = map_partition_varattnos(result, 1, rel, parent,
-									 &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");
-
-	/* Save a copy in the relcache */
-	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
-	rel->rd_partcheck = copyObject(result);
-	MemoryContextSwitchTo(oldcxt);
-
-	/* Keep the parent locked until commit */
-	heap_close(parent, NoLock);
-
-	return result;
-}
-
-/*
- * get_partition_for_tuple
- *		Finds partition of relation which accepts the partition key specified
- *		in values and isnull
- *
- * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
- * found or -1 if none found.
- */
-int
-get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
-{
-	int			bound_offset;
-	int			part_index = -1;
-	PartitionKey key = RelationGetPartitionKey(relation);
-	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
-
-	/* Route as appropriate based on partitioning strategy. */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				PartitionBoundInfo boundinfo = partdesc->boundinfo;
-				int			greatest_modulus = get_greatest_modulus(boundinfo);
-				uint64		rowHash = compute_hash_value(key, values, isnull);
-
-				part_index = boundinfo->indexes[rowHash % greatest_modulus];
-			}
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			if (isnull[0])
-			{
-				if (partition_bound_accepts_nulls(partdesc->boundinfo))
-					part_index = partdesc->boundinfo->null_index;
-			}
-			else
-			{
-				bool		equal = false;
-
-				bound_offset = partition_list_bsearch(key,
-													  partdesc->boundinfo,
-													  values[0], &equal);
-				if (bound_offset >= 0 && equal)
-					part_index = partdesc->boundinfo->indexes[bound_offset];
-			}
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				bool		equal = false,
-							range_partkey_has_null = false;
-				int			i;
-
-				/*
-				 * No range includes NULL, so this will be accepted by the
-				 * default partition if there is one, and otherwise rejected.
-				 */
-				for (i = 0; i < key->partnatts; i++)
-				{
-					if (isnull[i])
-					{
-						range_partkey_has_null = true;
-						break;
-					}
-				}
-
-				if (!range_partkey_has_null)
-				{
-					bound_offset = partition_range_datum_bsearch(key,
-														partdesc->boundinfo,
-														key->partnatts,
-														values,
-														&equal);
-					/*
-					 * The bound at bound_offset is less than or equal to the
-					 * tuple value, so the bound at offset+1 is the upper
-					 * bound of the partition we're looking for, if there
-					 * actually exists one.
-					 */
-					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
-				}
-			}
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/*
-	 * part_index < 0 means we failed to find a partition of this parent. Use
-	 * the default partition, if there is one.
-	 */
-	if (part_index < 0)
-		part_index = partdesc->boundinfo->default_index;
-
-	return part_index;
-}
-
-/*
- * Checks if any of the 'attnums' is a partition key attribute for rel
- *
- * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
- * partition key expression.  It's possible for a column to be both used
- * directly and as part of an expression; if that happens, *used_in_expr may
- * end up as either true or false.  That's OK for current uses of this
- * function, because *used_in_expr is only used to tailor the error message
- * text.
- */
-bool
-has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr)
-{
-	PartitionKey key;
-	int			partnatts;
-	List	   *partexprs;
-	ListCell   *partexprs_item;
-	int			i;
-
-	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return false;
-
-	key = RelationGetPartitionKey(rel);
-	partnatts = get_partition_natts(key);
-	partexprs = get_partition_exprs(key);
-
-	partexprs_item = list_head(partexprs);
-	for (i = 0; i < partnatts; i++)
-	{
-		AttrNumber	partattno = get_partition_col_attnum(key, i);
-
-		if (partattno != 0)
-		{
-			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
-							  attnums))
-			{
-				if (used_in_expr)
-					*used_in_expr = false;
-				return true;
-			}
-		}
-		else
-		{
-			/* Arbitrary expression */
-			Node	   *expr = (Node *) lfirst(partexprs_item);
-			Bitmapset  *expr_attrs = NULL;
-
-			/* Find all attributes referenced */
-			pull_varattnos(expr, 1, &expr_attrs);
-			partexprs_item = lnext(partexprs_item);
-
-			if (bms_overlap(attnums, expr_attrs))
-			{
-				if (used_in_expr)
-					*used_in_expr = true;
-				return true;
-			}
-		}
-	}
-
-	return false;
-}
-
-/*
- * qsort_partition_hbound_cmp
- *
- * We sort hash bounds by modulus, then by remainder.
- */
-static int32
-qsort_partition_hbound_cmp(const void *a, const void *b)
-{
-	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
-	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
-
-	return partition_hbound_cmp(h1->modulus, h1->remainder,
-								h2->modulus, h2->remainder);
-}
-
-/*
- * partition_hbound_cmp
- *
- * Compares modulus first, then remainder if modulus are equal.
- */
-static int32
-partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
-{
-	if (modulus1 < modulus2)
-		return -1;
-	if (modulus1 > modulus2)
-		return 1;
-	if (modulus1 == modulus2 && remainder1 != remainder2)
-		return (remainder1 > remainder2) ? 1 : -1;
-	return 0;
-}
-
-/*
- * qsort_partition_list_value_cmp
- *
- * Compare two list partition bound datums
- */
-static int32
-qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
-{
-	Datum		val1 = (*(const PartitionListValue **) a)->value,
-				val2 = (*(const PartitionListValue **) b)->value;
-	PartitionKey key = (PartitionKey) arg;
-
-	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-										   key->partcollation[0],
-										   val1, val2));
-}
-
-/*
- * make_one_range_bound
- *
- * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
- * and a flag telling whether the bound is lower or not.  Made into a function
- * because there are multiple sites that want to use this facility.
- */
-static PartitionRangeBound *
-make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
-{
-	PartitionRangeBound *bound;
-	ListCell   *lc;
-	int			i;
-
-	Assert(datums != NIL);
-
-	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
-	bound->index = index;
-	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
-	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
-													  sizeof(PartitionRangeDatumKind));
-	bound->lower = lower;
-
-	i = 0;
-	foreach(lc, datums)
-	{
-		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
-
-		/* What's contained in this range datum? */
-		bound->kind[i] = datum->kind;
-
-		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
-		{
-			Const	   *val = castNode(Const, datum->value);
-
-			if (val->constisnull)
-				elog(ERROR, "invalid range bound datum");
-			bound->datums[i] = val->constvalue;
-		}
-
-		i++;
-	}
-
-	return bound;
-}
-
-/* Used when sorting range bounds across all range partitions */
-static int32
-qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
-{
-	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
-	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
-	PartitionKey key = (PartitionKey) arg;
-
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
-}
-
-/*
- * partition_rbound_cmp
- *
- * Return for two range bounds whether the 1st one (specified in datums1,
- * kind1, and lower1) is <, =, or > the bound specified in *b2.
- *
- * Note that if the values of the two range bounds compare equal, then we take
- * into account whether they are upper or lower bounds, and an upper bound is
- * considered to be smaller than a lower bound. This is important to the way
- * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
- * structure, which only stores the upper bound of a common boundary between
- * two contiguous partitions.
- */
-static int32
-partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2)
-{
-	int32		cmpval = 0;		/* placate compiler */
-	int			i;
-	Datum	   *datums2 = b2->datums;
-	PartitionRangeDatumKind *kind2 = b2->kind;
-	bool		lower2 = b2->lower;
-
-	for (i = 0; i < key->partnatts; i++)
-	{
-		/*
-		 * First, handle cases where the column is unbounded, which should not
-		 * invoke the comparison procedure, and should not consider any later
-		 * columns. Note that the PartitionRangeDatumKind enum elements
-		 * compare the same way as the values they represent.
-		 */
-		if (kind1[i] < kind2[i])
-			return -1;
-		else if (kind1[i] > kind2[i])
-			return 1;
-		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
-
-			/*
-			 * The column bounds are both MINVALUE or both MAXVALUE. No later
-			 * columns should be considered, but we still need to compare
-			 * whether they are upper or lower bounds.
-			 */
-			break;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 datums1[i],
-												 datums2[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	/*
-	 * If the comparison is anything other than equal, we're done. If they
-	 * compare equal though, we still have to consider whether the boundaries
-	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
-	 * two.
-	 */
-	if (cmpval == 0 && lower1 != lower2)
-		cmpval = lower1 ? 1 : -1;
-
-	return cmpval;
-}
-
-/*
- * partition_rbound_datum_cmp
- *
- * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
- * is <, =, or > partition key of tuple (tuple_datums)
- */
-static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums)
-{
-	int			i;
-	int32		cmpval = -1;
-
-	for (i = 0; i < n_tuple_datums; i++)
-	{
-		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
-			return -1;
-		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
-			return 1;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 rb_datums[i],
-												 tuple_datums[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	return cmpval;
-}
-
-/*
- * partition_list_bsearch
- *		Returns the index of the greatest bound datum that is less than equal
- * 		to the given value or -1 if all of the bound datums are greater
- *
- * *is_equal is set to true if the bound datum at the returned index is equal
- * to the input value.
- */
-static int
-partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-												 key->partcollation[0],
-												 boundinfo->datums[mid][0],
-												 value));
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given range bound or -1 if all of the range bounds are
- *		greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input range bound
- */
-static int
-partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_cmp(key,
-									  boundinfo->datums[mid],
-									  boundinfo->kind[mid],
-									  (boundinfo->indexes[mid] == -1),
-									  probe);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given tuple or -1 if all of the range bounds are greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input tuple.
- */
-static int
-partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_datum_cmp(key,
-											boundinfo->datums[mid],
-											boundinfo->kind[mid],
-											values,
-											nvalues);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_hash_bsearch
- *		Returns the index of the greatest (modulus, remainder) pair that is
- *		less than or equal to the given (modulus, remainder) pair or -1 if
- *		all of them are greater
- */
-static int
-partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval,
-					bound_modulus,
-					bound_remainder;
-
-		mid = (lo + hi + 1) / 2;
-		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
-		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
-		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
-									  modulus, remainder);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-
-			if (cmpval == 0)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * get_default_oid_from_partdesc
- *
- * Given a partition descriptor, return the OID of the default partition, if
- * one exists; else, return InvalidOid.
- */
-Oid
-get_default_oid_from_partdesc(PartitionDesc partdesc)
-{
-	if (partdesc && partdesc->boundinfo &&
-		partition_bound_has_default(partdesc->boundinfo))
-		return partdesc->oids[partdesc->boundinfo->default_index];
-
-	return InvalidOid;
-}
-
-/*
  * get_default_partition_oid
  *
  * Given a relation OID, return the OID of the default partition, if one
@@ -3189,99 +1305,6 @@ get_proposed_default_constraint(List *new_part_constraints)
 }
 
 /*
- * get_partition_bound_num_indexes
- *
- * Returns the number of the entries in the partition bound indexes array.
- */
-static int
-get_partition_bound_num_indexes(PartitionBoundInfo bound)
-{
-	int			num_indexes;
-
-	Assert(bound);
-
-	switch (bound->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-
-			/*
-			 * The number of the entries in the indexes array is same as the
-			 * greatest modulus.
-			 */
-			num_indexes = get_greatest_modulus(bound);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			num_indexes = bound->ndatums;
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			/* Range partitioned table has an extra index. */
-			num_indexes = bound->ndatums + 1;
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) bound->strategy);
-	}
-
-	return num_indexes;
-}
-
-/*
- * get_greatest_modulus
- *
- * Returns the greatest modulus of the hash partition bound. The greatest
- * modulus will be at the end of the datums array because hash partitions are
- * arranged in the ascending order of their modulus and remainders.
- */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
-{
-	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
-	Assert(bound->datums && bound->ndatums > 0);
-	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
-
-	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
-}
-
-/*
- * compute_hash_value
- *
- * Compute the hash value for given not null partition key values.
- */
-static uint64
-compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
-{
-	int			i;
-	int			nkeys = key->partnatts;
-	uint64		rowHash = 0;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-
-	for (i = 0; i < nkeys; i++)
-	{
-		if (!isnull[i])
-		{
-			Datum		hash;
-
-			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
-
-			/*
-			 * Compute hash for each datum value by calling respective
-			 * datatype-specific hash functions of each partition key
-			 * attribute.
-			 */
-			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	return rowHash;
-}
-
-/*
  * satisfies_hash_partition
  *
  * This is an SQL-callable function for use in hash partition constraints.
diff --git a/src/backend/executor/execExpr.c b/src/backend/executor/execExpr.c
index c6eb3ebacf..6c9d89523d 100644
--- a/src/backend/executor/execExpr.c
+++ b/src/backend/executor/execExpr.c
@@ -42,6 +42,7 @@
 #include "optimizer/clauses.h"
 #include "optimizer/planner.h"
 #include "pgstat.h"
+#include "utils/acl.h"
 #include "utils/builtins.h"
 #include "utils/datum.h"
 #include "utils/lsyscache.h"
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 5d3e923cca..b17abb5c7d 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -42,7 +42,6 @@
 #include "access/transam.h"
 #include "access/xact.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_publication.h"
 #include "commands/matview.h"
 #include "commands/trigger.h"
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 4048c3ebc6..cc77ba3701 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -20,6 +20,7 @@
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2a8ecbd830..a4a8326ad3 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -46,6 +46,7 @@
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
+#include "rewrite/rewriteManip.h"
 #include "storage/bufmgr.h"
 #include "storage/lmgr.h"
 #include "utils/builtins.h"
diff --git a/src/backend/libpq/hba.c b/src/backend/libpq/hba.c
index acf625e4ec..f930f62517 100644
--- a/src/backend/libpq/hba.c
+++ b/src/backend/libpq/hba.c
@@ -26,6 +26,7 @@
 #include <unistd.h>
 
 #include "access/htup_details.h"
+#include "catalog/objectaddress.h"
 #include "catalog/pg_collation.h"
 #include "catalog/pg_type.h"
 #include "common/ip.h"
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index f74afdb4dd..7164a09995 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -15,13 +15,13 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/prep.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/memutils.h"
 
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b586f941a8..8c7caabbc7 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,7 +33,6 @@
 #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"
@@ -49,6 +48,7 @@
 #include "parser/parse_coerce.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
 
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 60f21711f4..605493d04a 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,7 +27,6 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_statistic_ext.h"
 #include "foreign/fdwapi.h"
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 5c368321e6..74f8e0a04d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -17,7 +17,6 @@
 #include <limits.h>
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
@@ -28,6 +27,7 @@
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
+#include "utils/partcache.h"
 
 
 typedef struct JoinHashEntry
diff --git a/src/backend/rewrite/rewriteManip.c b/src/backend/rewrite/rewriteManip.c
index abad1bf7e4..c0982a0cfe 100644
--- a/src/backend/rewrite/rewriteManip.c
+++ b/src/backend/rewrite/rewriteManip.c
@@ -13,6 +13,7 @@
  */
 #include "postgres.h"
 
+#include "access/tupconvert.h"
 #include "catalog/pg_type.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -1374,6 +1375,50 @@ map_variable_attnos(Node *node,
 											0);
 }
 
+/*
+ * map_partition_varattnos - maps varattno of any Vars in expr from the
+ * attno's of 'from_rel' to the attno's of 'to_rel' partition, each of which
+ * may be either a leaf partition or a partitioned table, but both of which
+ * must be from the same partitioning hierarchy.
+ *
+ * Even though all of the same column names must be present in all relations
+ * in the hierarchy, and they must also have the same types, the attnos may
+ * be different.
+ *
+ * If found_whole_row is not NULL, *found_whole_row returns whether a
+ * whole-row variable was found in the input expression.
+ *
+ * Note: this will work on any node tree, so really the argument and result
+ * should be declared "Node *".  But a substantial majority of the callers
+ * are working on Lists, so it's less messy to do the casts internally.
+ */
+List *
+map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel,
+						bool *found_whole_row)
+{
+	bool		my_found_whole_row = false;
+
+	if (expr != NIL)
+	{
+		AttrNumber *part_attnos;
+
+		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,
+											fromrel_varno, 0,
+											part_attnos,
+											RelationGetDescr(from_rel)->natts,
+											RelationGetForm(to_rel)->reltype,
+											&my_found_whole_row);
+	}
+
+	if (found_whole_row)
+		*found_whole_row = my_found_whole_row;
+
+	return expr;
+}
 
 /*
  * ReplaceVarsFromTargetList - replace Vars with items from a targetlist
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3abe7d6155..beb3c0c44a 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -67,6 +67,7 @@
 #include "tcop/utility.h"
 #include "utils/acl.h"
 #include "utils/guc.h"
+#include "utils/rel.h"
 #include "utils/syscache.h"
 
 
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index ba9fab4582..0797e258b3 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -24,7 +24,6 @@
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
-#include "catalog/partition.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_authid.h"
diff --git a/src/backend/utils/cache/Makefile b/src/backend/utils/cache/Makefile
index a943f8ea4b..94511eaf54 100644
--- a/src/backend/utils/cache/Makefile
+++ b/src/backend/utils/cache/Makefile
@@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
-	relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
-	typcache.o ts_cache.o
+OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o partcache.o \
+    relcache.o relmapper.o relfilenodemap.o spccache.o syscache.o \
+    lsyscache.o typcache.o ts_cache.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
new file mode 100644
index 0000000000..94b9b52547
--- /dev/null
+++ b/src/backend/utils/cache/partcache.c
@@ -0,0 +1,2118 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.c
+ *		Partitioning related cache data structures and manipulation functions
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *		  src/backend/utils/cache/partcache.c
+ *
+ *-------------------------------------------------------------------------
+*/
+
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "access/nbtree.h"
+#include "access/sysattr.h"
+#include "catalog/partition.h"
+#include "catalog/pg_inherits_fn.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/var.h"
+#include "nodes/parsenodes.h"
+#include "rewrite/rewriteManip.h"
+#include "utils/builtins.h"
+#include "utils/datum.h"
+#include "utils/hashutils.h"
+#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
+#include "utils/ruleutils.h"
+#include "utils/syscache.h"
+
+/*
+ * Partition bounds are put into using one of the following structs when they
+ * are first read into the backend memory from the catalog wherein they are
+ * stored in in their parser represenation.  Especially, the bound comparison
+ * and sort functions expect the bounds passed to them to be of this form.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int			modulus;
+	int			remainder;
+	int			index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+	int			index;
+	Datum		value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+	int			index;
+	Datum	   *datums;			/* range bound datums */
+	PartitionRangeDatumKind *kind;	/* the kind of each datum */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+static List *generate_partition_qual(Relation rel);
+
+static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
+					 int remainder2);
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
+static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
+							   void *arg);
+static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
+					 List *datums, bool lower);
+static int32 partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2);
+static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
+						   void *arg);
+
+static int partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal);
+static int partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal);
+static int32 partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums);
+static int partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal);
+static int partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder);
+
+static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
+
+/*
+ * RelationBuildPartitionKey
+ *		Build and attach to relcache partition key data of relation
+ *
+ * Partitioning key data is a complex structure; to avoid complicated logic to
+ * free individual elements whenever the relcache entry is flushed, we give it
+ * its own memory context, child of CacheMemoryContext, which can easily be
+ * deleted on its own.  To avoid leaking memory in that context in case of an
+ * error partway through this function, the context is initially created as a
+ * child of CurTransactionContext and only re-parented to CacheMemoryContext
+ * at the end, when no further errors are possible.  Also, we don't make this
+ * context the current context except in very brief code sections, out of fear
+ * that some of our callees allocate memory on their own which would be leaked
+ * permanently.
+ */
+void
+RelationBuildPartitionKey(Relation relation)
+{
+	Form_pg_partitioned_table form;
+	HeapTuple	tuple;
+	bool		isnull;
+	int			i;
+	PartitionKey key;
+	AttrNumber *attrs;
+	oidvector  *opclass;
+	oidvector  *collation;
+	ListCell   *partexprs_item;
+	Datum		datum;
+	MemoryContext partkeycxt,
+				oldcxt;
+	int16		procnum;
+
+	tuple = SearchSysCache1(PARTRELID,
+							ObjectIdGetDatum(RelationGetRelid(relation)));
+
+	/*
+	 * The following happens when we have created our pg_class entry but not
+	 * the pg_partitioned_table entry yet.
+	 */
+	if (!HeapTupleIsValid(tuple))
+		return;
+
+	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
+											   RelationGetRelationName(relation),
+											   MEMCONTEXT_COPY_NAME,
+											   ALLOCSET_SMALL_SIZES);
+
+	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
+												sizeof(PartitionKeyData));
+
+	/* Fixed-length attributes */
+	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	key->strategy = form->partstrat;
+	key->partnatts = form->partnatts;
+
+	/*
+	 * We can rely on the first variable-length attribute being mapped to the
+	 * relevant field of the catalog's C struct, because all previous
+	 * attributes are non-nullable and fixed-length.
+	 */
+	attrs = form->partattrs.values;
+
+	/* But use the hard way to retrieve further variable-length attributes */
+	/* Operator class */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partclass, &isnull);
+	Assert(!isnull);
+	opclass = (oidvector *) DatumGetPointer(datum);
+
+	/* Collation */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partcollation, &isnull);
+	Assert(!isnull);
+	collation = (oidvector *) DatumGetPointer(datum);
+
+	/* Expressions */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partexprs, &isnull);
+	if (!isnull)
+	{
+		char	   *exprString;
+		Node	   *expr;
+
+		exprString = TextDatumGetCString(datum);
+		expr = stringToNode(exprString);
+		pfree(exprString);
+
+		/*
+		 * Run the expressions through const-simplification since the planner
+		 * will be comparing them to similarly-processed qual clause operands,
+		 * and may fail to detect valid matches without this step; fix
+		 * opfuncids while at it.  We don't need to bother with
+		 * canonicalize_qual() though, because partition expressions are not
+		 * full-fledged qualification clauses.
+		 */
+		expr = eval_const_expressions(NULL, expr);
+		fix_opfuncids(expr);
+
+		oldcxt = MemoryContextSwitchTo(partkeycxt);
+		key->partexprs = (List *) copyObject(expr);
+		MemoryContextSwitchTo(oldcxt);
+	}
+
+	oldcxt = MemoryContextSwitchTo(partkeycxt);
+	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
+	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
+
+	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+
+	/* Gather type and collation info as well */
+	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
+	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
+	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
+	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
+	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	MemoryContextSwitchTo(oldcxt);
+
+	/* determine support function number to search for */
+	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
+		HASHEXTENDED_PROC : BTORDER_PROC;
+
+	/* Copy partattrs and fill other per-attribute info */
+	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		AttrNumber	attno = key->partattrs[i];
+		HeapTuple	opclasstup;
+		Form_pg_opclass opclassform;
+		Oid			funcid;
+
+		/* Collect opfamily information */
+		opclasstup = SearchSysCache1(CLAOID,
+									 ObjectIdGetDatum(opclass->values[i]));
+		if (!HeapTupleIsValid(opclasstup))
+			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
+
+		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
+		key->partopfamily[i] = opclassform->opcfamily;
+		key->partopcintype[i] = opclassform->opcintype;
+
+		/* Get a support function for the specified opfamily and datatypes */
+		funcid = get_opfamily_proc(opclassform->opcfamily,
+								   opclassform->opcintype,
+								   opclassform->opcintype,
+								   procnum);
+		if (!OidIsValid(funcid))
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
+							NameStr(opclassform->opcname),
+							(key->strategy == PARTITION_STRATEGY_HASH) ?
+							"hash" : "btree",
+							procnum,
+							format_type_be(opclassform->opcintype))));
+
+		fmgr_info(funcid, &key->partsupfunc[i]);
+
+		/* Collation */
+		key->partcollation[i] = collation->values[i];
+
+		/* Collect type information */
+		if (attno != 0)
+		{
+			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
+
+			key->parttypid[i] = att->atttypid;
+			key->parttypmod[i] = att->atttypmod;
+			key->parttypcoll[i] = att->attcollation;
+		}
+		else
+		{
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			key->parttypid[i] = exprType(lfirst(partexprs_item));
+			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
+			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
+
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(key->parttypid[i],
+							 &key->parttyplen[i],
+							 &key->parttypbyval[i],
+							 &key->parttypalign[i]);
+
+		ReleaseSysCache(opclasstup);
+	}
+
+	ReleaseSysCache(tuple);
+
+	/*
+	 * Success --- reparent our context and make the relcache point to the
+	 * newly constructed key
+	 */
+	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
+	relation->rd_partkeycxt = partkeycxt;
+	relation->rd_partkey = key;
+}
+
+/*
+ * RelationBuildPartitionDesc
+ *		Form rel's partition descriptor
+ *
+ * Not flushed from the cache by RelationClearRelation() unless changed because
+ * of addition or removal of partition.
+ */
+void
+RelationBuildPartitionDesc(Relation rel)
+{
+	List	   *inhoids,
+			   *partoids;
+	Oid		   *oids = NULL;
+	List	   *boundspecs = NIL;
+	ListCell   *cell;
+	int			i,
+				nparts;
+	PartitionKey key = RelationGetPartitionKey(rel);
+	PartitionDesc result;
+	MemoryContext oldcxt;
+
+	int			ndatums = 0;
+	int			default_index = -1;
+
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
+	/* List partitioning specific */
+	PartitionListValue **all_values = NULL;
+	int			null_index = -1;
+
+	/* Range partitioning specific */
+	PartitionRangeBound **rbounds = NULL;
+
+	/*
+	 * The following could happen in situations where rel has a pg_class entry
+	 * but not the pg_partitioned_table entry yet.
+	 */
+	if (key == NULL)
+		return;
+
+	/* Get partition oids from pg_inherits */
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+
+	/* Collect bound spec nodes in a list */
+	i = 0;
+	partoids = NIL;
+	foreach(cell, inhoids)
+	{
+		Oid			inhrelid = lfirst_oid(cell);
+		HeapTuple	tuple;
+		Datum		datum;
+		bool		isnull;
+		Node	   *boundspec;
+
+		tuple = SearchSysCache1(RELOID, inhrelid);
+		if (!HeapTupleIsValid(tuple))
+			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+		/*
+		 * It is possible that the pg_class tuple of a partition has not been
+		 * updated yet to set its relpartbound field.  The only case where
+		 * this happens is when we open the parent relation to check using its
+		 * partition descriptor that a new partition's bound does not overlap
+		 * some existing partition.
+		 */
+		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		{
+			ReleaseSysCache(tuple);
+			continue;
+		}
+
+		datum = SysCacheGetAttr(RELOID, tuple,
+								Anum_pg_class_relpartbound,
+								&isnull);
+		Assert(!isnull);
+		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+		/*
+		 * Sanity check: If the PartitionBoundSpec says this is the default
+		 * partition, its OID should correspond to whatever's stored in
+		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 */
+		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		{
+			Oid			partdefid;
+
+			partdefid = get_default_partition_oid(RelationGetRelid(rel));
+			if (partdefid != inhrelid)
+				elog(ERROR, "expected partdefid %u, but got %u",
+					 inhrelid, partdefid);
+		}
+
+		boundspecs = lappend(boundspecs, boundspec);
+		partoids = lappend_oid(partoids, inhrelid);
+		ReleaseSysCache(tuple);
+	}
+
+	nparts = list_length(partoids);
+
+	if (nparts > 0)
+	{
+		oids = (Oid *) palloc(nparts * sizeof(Oid));
+		i = 0;
+		foreach(cell, partoids)
+			oids[i++] = lfirst_oid(cell);
+
+		/* Convert from node to the internal representation */
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **)
+				palloc(nparts * sizeof(PartitionHashBound *));
+
+			i = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
+		{
+			List	   *non_null_values = NIL;
+
+			/*
+			 * Create a unified list of non-null values across all partitions.
+			 */
+			i = 0;
+			null_index = -1;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				ListCell   *c;
+
+				if (spec->strategy != PARTITION_STRATEGY_LIST)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
+				foreach(c, spec->listdatums)
+				{
+					Const	   *val = castNode(Const, lfirst(c));
+					PartitionListValue *list_value = NULL;
+
+					if (!val->constisnull)
+					{
+						list_value = (PartitionListValue *)
+							palloc0(sizeof(PartitionListValue));
+						list_value->index = i;
+						list_value->value = val->constvalue;
+					}
+					else
+					{
+						/*
+						 * Never put a null into the values array, flag
+						 * instead for the code further down below where we
+						 * construct the actual relcache struct.
+						 */
+						if (null_index != -1)
+							elog(ERROR, "found null more than once");
+						null_index = i;
+					}
+
+					if (list_value)
+						non_null_values = lappend(non_null_values,
+												  list_value);
+				}
+
+				i++;
+			}
+
+			ndatums = list_length(non_null_values);
+
+			/*
+			 * Collect all list values in one array. Alongside the value, we
+			 * also save the index of partition the value comes from.
+			 */
+			all_values = (PartitionListValue **) palloc(ndatums *
+														sizeof(PartitionListValue *));
+			i = 0;
+			foreach(cell, non_null_values)
+			{
+				PartitionListValue *src = lfirst(cell);
+
+				all_values[i] = (PartitionListValue *)
+					palloc(sizeof(PartitionListValue));
+				all_values[i]->value = src->value;
+				all_values[i]->index = src->index;
+				i++;
+			}
+
+			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
+					  qsort_partition_list_value_cmp, (void *) key);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_RANGE)
+		{
+			int			k;
+			PartitionRangeBound **all_bounds,
+					   *prev;
+
+			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
+														  sizeof(PartitionRangeBound *));
+
+			/*
+			 * Create a unified list of range bounds across all the
+			 * partitions.
+			 */
+			i = ndatums = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				PartitionRangeBound *lower,
+						   *upper;
+
+				if (spec->strategy != PARTITION_STRATEGY_RANGE)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the allbounds array
+				 * for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i++;
+					continue;
+				}
+
+				lower = make_one_range_bound(key, i, spec->lowerdatums,
+											 true);
+				upper = make_one_range_bound(key, i, spec->upperdatums,
+											 false);
+				all_bounds[ndatums++] = lower;
+				all_bounds[ndatums++] = upper;
+				i++;
+			}
+
+			Assert(ndatums == nparts * 2 ||
+				   (default_index != -1 && ndatums == (nparts - 1) * 2));
+
+			/* Sort all the bounds in ascending order */
+			qsort_arg(all_bounds, ndatums,
+					  sizeof(PartitionRangeBound *),
+					  qsort_partition_rbound_cmp,
+					  (void *) key);
+
+			/* Save distinct bounds from all_bounds into rbounds. */
+			rbounds = (PartitionRangeBound **)
+				palloc(ndatums * sizeof(PartitionRangeBound *));
+			k = 0;
+			prev = NULL;
+			for (i = 0; i < ndatums; i++)
+			{
+				PartitionRangeBound *cur = all_bounds[i];
+				bool		is_distinct = false;
+				int			j;
+
+				/* Is the current bound distinct from the previous one? */
+				for (j = 0; j < key->partnatts; j++)
+				{
+					Datum		cmpval;
+
+					if (prev == NULL || cur->kind[j] != prev->kind[j])
+					{
+						is_distinct = true;
+						break;
+					}
+
+					/*
+					 * If the bounds are both MINVALUE or MAXVALUE, stop now
+					 * and treat them as equal, since any values after this
+					 * point must be ignored.
+					 */
+					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
+						break;
+
+					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
+											   key->partcollation[j],
+											   cur->datums[j],
+											   prev->datums[j]);
+					if (DatumGetInt32(cmpval) != 0)
+					{
+						is_distinct = true;
+						break;
+					}
+				}
+
+				/*
+				 * Only if the bound is distinct save it into a temporary
+				 * array i.e. rbounds which is later copied into boundinfo
+				 * datums array.
+				 */
+				if (is_distinct)
+					rbounds[k++] = all_bounds[i];
+
+				prev = cur;
+			}
+
+			/* Update ndatums to hold the count of distinct datums. */
+			ndatums = k;
+		}
+		else
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/* Now build the actual relcache partition descriptor */
+	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
+												  RelationGetRelationName(rel),
+												  MEMCONTEXT_COPY_NAME,
+												  ALLOCSET_DEFAULT_SIZES);
+	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+
+	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
+	result->nparts = nparts;
+	if (nparts > 0)
+	{
+		PartitionBoundInfo boundinfo;
+		int		   *mapping;
+		int			next_index = 0;
+
+		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
+
+		boundinfo = (PartitionBoundInfoData *)
+			palloc0(sizeof(PartitionBoundInfoData));
+		boundinfo->strategy = key->strategy;
+		boundinfo->default_index = -1;
+		boundinfo->ndatums = ndatums;
+		boundinfo->null_index = -1;
+		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
+
+		/* Initialize mapping array with invalid values */
+		mapping = (int *) palloc(sizeof(int) * nparts);
+		for (i = 0; i < nparts; i++)
+			mapping[i] = -1;
+
+		switch (key->strategy)
+		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+
+					for (i = 0; i < greatest_modulus; i++)
+						boundinfo->indexes[i] = -1;
+
+					for (i = 0; i < nparts; i++)
+					{
+						int			modulus = hbounds[i]->modulus;
+						int			remainder = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(modulus);
+						boundinfo->datums[i][1] = Int32GetDatum(remainder);
+
+						while (remainder < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[remainder] == -1);
+							boundinfo->indexes[remainder] = i;
+							remainder += modulus;
+						}
+
+						mapping[hbounds[i]->index] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
+			case PARTITION_STRATEGY_LIST:
+				{
+					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
+
+					/*
+					 * Copy values.  Indexes of individual values are mapped
+					 * to canonical values so that they match for any two list
+					 * partitioned tables with same number of partitions and
+					 * same lists per partition.  One way to canonicalize is
+					 * to assign the index in all_values[] of the smallest
+					 * value of each partition, as the index of all of the
+					 * partition's values.
+					 */
+					for (i = 0; i < ndatums; i++)
+					{
+						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
+						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
+															key->parttypbyval[0],
+															key->parttyplen[0]);
+
+						/* If the old index has no mapping, assign one */
+						if (mapping[all_values[i]->index] == -1)
+							mapping[all_values[i]->index] = next_index++;
+
+						boundinfo->indexes[i] = mapping[all_values[i]->index];
+					}
+
+					/*
+					 * If null-accepting partition has no mapped index yet,
+					 * assign one.  This could happen if such partition
+					 * accepts only null and hence not covered in the above
+					 * loop which only handled non-null values.
+					 */
+					if (null_index != -1)
+					{
+						Assert(null_index >= 0);
+						if (mapping[null_index] == -1)
+							mapping[null_index] = next_index++;
+						boundinfo->null_index = mapping[null_index];
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any value not
+						 * specified in the lists of other partitions, hence
+						 * it should not get mapped index while assigning
+						 * those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+
+					/* All partition must now have a valid mapping */
+					Assert(next_index == nparts);
+					break;
+				}
+
+			case PARTITION_STRATEGY_RANGE:
+				{
+					boundinfo->kind = (PartitionRangeDatumKind **)
+						palloc(ndatums *
+							   sizeof(PartitionRangeDatumKind *));
+					boundinfo->indexes = (int *) palloc((ndatums + 1) *
+														sizeof(int));
+
+					for (i = 0; i < ndatums; i++)
+					{
+						int			j;
+
+						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
+																sizeof(Datum));
+						boundinfo->kind[i] = (PartitionRangeDatumKind *)
+							palloc(key->partnatts *
+								   sizeof(PartitionRangeDatumKind));
+						for (j = 0; j < key->partnatts; j++)
+						{
+							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
+								boundinfo->datums[i][j] =
+									datumCopy(rbounds[i]->datums[j],
+											  key->parttypbyval[j],
+											  key->parttyplen[j]);
+							boundinfo->kind[i][j] = rbounds[i]->kind[j];
+						}
+
+						/*
+						 * There is no mapping for invalid indexes.
+						 *
+						 * Any lower bounds in the rbounds array have invalid
+						 * indexes assigned, because the values between the
+						 * previous bound (if there is one) and this (lower)
+						 * bound are not part of the range of any existing
+						 * partition.
+						 */
+						if (rbounds[i]->lower)
+							boundinfo->indexes[i] = -1;
+						else
+						{
+							int			orig_index = rbounds[i]->index;
+
+							/* If the old index has no mapping, assign one */
+							if (mapping[orig_index] == -1)
+								mapping[orig_index] = next_index++;
+
+							boundinfo->indexes[i] = mapping[orig_index];
+						}
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						Assert(default_index >= 0 && mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					boundinfo->indexes[i] = -1;
+					break;
+				}
+
+			default:
+				elog(ERROR, "unexpected partition strategy: %d",
+					 (int) key->strategy);
+		}
+
+		result->boundinfo = boundinfo;
+
+		/*
+		 * Now assign OIDs from the original array into mapped indexes of the
+		 * result array.  Order of OIDs in the former is defined by the
+		 * catalog scan that retrieved them, whereas that in the latter is
+		 * defined by canonicalized representation of the partition bounds.
+		 */
+		for (i = 0; i < nparts; i++)
+			result->oids[mapping[i]] = oids[i];
+		pfree(mapping);
+	}
+
+	MemoryContextSwitchTo(oldcxt);
+	rel->rd_partdesc = result;
+}
+
+/*
+ * Are two partition bound collections logically equal?
+ *
+ * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
+ * This is also useful when b1 and b2 are bound collections of two separate
+ * relations, respectively, because PartitionBoundInfo is a canonical
+ * representation of partition bounds.
+ */
+bool
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
+					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			i;
+
+	if (b1->strategy != b2->strategy)
+		return false;
+
+	if (b1->ndatums != b2->ndatums)
+		return false;
+
+	if (b1->null_index != b2->null_index)
+		return false;
+
+	if (b1->default_index != b2->default_index)
+		return false;
+
+	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	{
+		int			greatest_modulus = get_greatest_modulus(b1);
+
+		/*
+		 * If two hash partitioned tables have different greatest moduli,
+		 * their partition schemes don't match.
+		 */
+		if (greatest_modulus != get_greatest_modulus(b2))
+			return false;
+
+		/*
+		 * We arrange the partitions in the ascending order of their modulus
+		 * and remainders.  Also every modulus is factor of next larger
+		 * modulus.  Therefore we can safely store index of a given partition
+		 * in indexes array at remainder of that partition.  Also entries at
+		 * (remainder + N * modulus) positions in indexes array are all same
+		 * for (modulus, remainder) specification for any partition.  Thus
+		 * datums array from both the given bounds are same, if and only if
+		 * their indexes array will be same.  So, it suffices to compare
+		 * indexes array.
+		 */
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Nonetheless make sure that the bounds are indeed same when the
+		 * indexes match.  Hash partition bound stores modulus and remainder
+		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			Assert((b1->datums[i][0] == b2->datums[i][0] &&
+					b1->datums[i][1] == b2->datums[i][1]));
+#endif
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
+		{
+			int			j;
+
+			for (j = 0; j < partnatts; j++)
+			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->kind != NULL)
+				{
+					/* The different kinds of bound all differ from each other */
+					if (b1->kind[i][j] != b2->kind[i][j])
+						return false;
+
+					/*
+					 * Non-finite bounds are equal without further
+					 * examination.
+					 */
+					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
+						continue;
+				}
+
+				/*
+				 * Compare the actual values. Note that it would be both
+				 * incorrect and unsafe to invoke the comparison operator
+				 * derived from the partitioning specification here.  It would
+				 * be incorrect because we want the relcache entry to be
+				 * updated for ANY change to the partition bounds, not just
+				 * those that the partitioning operator thinks are
+				 * significant.  It would be unsafe because we might reach
+				 * this code in the context of an aborted transaction, and an
+				 * arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be
+				 * safe.
+				 */
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  parttypbyval[j], parttyplen[j]))
+					return false;
+			}
+
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+		}
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
+			return false;
+	}
+	return true;
+}
+
+/*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specification.
+ */
+PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo dest;
+	int			i;
+	int			ndatums;
+	int			partnatts;
+	int			num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	num_indexes = get_partition_bound_num_indexes(src);
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+														 sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+															   sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int			j;
+
+		/*
+		 * For a corresponding to hash partition, datums array will have two
+		 * elements - modulus and remainder.
+		 */
+		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
+		int			natts = hash_part ? 2 : partnatts;
+
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+
+		for (j = 0; j < natts; j++)
+		{
+			bool		byval;
+			int			typlen;
+
+			if (hash_part)
+			{
+				typlen = sizeof(int32); /* Always int4 */
+				byval = true;	/* int4 is pass-by-value */
+			}
+			else
+			{
+				byval = key->parttypbyval[j];
+				typlen = key->parttyplen[j];
+			}
+
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   byval, typlen);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
+ * check_new_partition_bound
+ *
+ * Checks if the new partition's bound overlaps any of the existing partitions
+ * of parent.  Also performs additional checks as necessary per strategy.
+ */
+void
+check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
+	ParseState *pstate = make_parsestate(NULL);
+	int			with = -1;
+	bool		overlap = false;
+
+	if (spec->is_default)
+	{
+		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+			return;
+
+		/* Default partition already exists, error out. */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+				 parser_errposition(pstate, spec->location)));
+	}
+
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					Datum	  **datums = boundinfo->datums;
+					int			ndatums = boundinfo->ndatums;
+					int			greatest_modulus;
+					int			remainder;
+					int			offset;
+					bool		valid_modulus = true;
+					int			prev_modulus,	/* Previous largest modulus */
+								next_modulus;	/* Next largest modulus */
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.
+					 *
+					 * Get the greatest (modulus, remainder) pair contained in
+					 * boundinfo->datums that is less than or equal to the
+					 * (spec->modulus, spec->remainder) pair.
+					 */
+					offset = partition_hash_bsearch(key, boundinfo,
+													spec->modulus,
+													spec->remainder);
+					if (offset < 0)
+					{
+						next_modulus = DatumGetInt32(datums[0][0]);
+						valid_modulus = (next_modulus % spec->modulus) == 0;
+					}
+					else
+					{
+						prev_modulus = DatumGetInt32(datums[offset][0]);
+						valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+						if (valid_modulus && (offset + 1) < ndatums)
+						{
+							next_modulus = DatumGetInt32(datums[offset + 1][0]);
+							valid_modulus = (next_modulus % spec->modulus) == 0;
+						}
+					}
+
+					if (!valid_modulus)
+						ereport(ERROR,
+								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
+
+					greatest_modulus = get_greatest_modulus(boundinfo);
+					remainder = spec->remainder;
+
+					/*
+					 * Normally, the lowest remainder that could conflict with
+					 * the new partition is equal to the remainder specified
+					 * for the new partition, but when the new partition has a
+					 * modulus higher than any used so far, we need to adjust.
+					 */
+					if (remainder >= greatest_modulus)
+						remainder = remainder % greatest_modulus;
+
+					/* Check every potentially-conflicting remainder. */
+					do
+					{
+						if (boundinfo->indexes[remainder] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[remainder];
+							break;
+						}
+						remainder += spec->modulus;
+					} while (remainder < greatest_modulus);
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_LIST:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+
+				if (partdesc->nparts > 0)
+				{
+					ListCell   *cell;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					foreach(cell, spec->listdatums)
+					{
+						Const	   *val = castNode(Const, lfirst(cell));
+
+						if (!val->constisnull)
+						{
+							int			offset;
+							bool		equal;
+
+							offset = partition_list_bsearch(key, boundinfo,
+															val->constvalue,
+															&equal);
+							if (offset >= 0 && equal)
+							{
+								overlap = true;
+								with = boundinfo->indexes[offset];
+								break;
+							}
+						}
+						else if (partition_bound_accepts_nulls(boundinfo))
+						{
+							overlap = true;
+							with = boundinfo->null_index;
+							break;
+						}
+					}
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				PartitionRangeBound *lower,
+						   *upper;
+
+				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
+				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
+
+				/*
+				 * First check if the resulting range would be empty with
+				 * specified lower and upper bounds
+				 */
+				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
+										 upper) >= 0)
+				{
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("empty range bound specified for partition \"%s\"",
+									relname),
+							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
+									   get_range_partbound_string(spec->lowerdatums),
+									   get_range_partbound_string(spec->upperdatums)),
+							 parser_errposition(pstate, spec->location)));
+				}
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					int			offset;
+					bool		equal;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Test whether the new lower bound (which is treated
+					 * inclusively as part of the new partition) lies inside
+					 * an existing partition, or in a gap.
+					 *
+					 * If it's inside an existing partition, the bound at
+					 * offset + 1 will be the upper bound of that partition,
+					 * and its index will be >= 0.
+					 *
+					 * If it's in a gap, the bound at offset + 1 will be the
+					 * lower bound of the next partition, and its index will
+					 * be -1. This is also true if there is no next partition,
+					 * since the index array is initialised with an extra -1
+					 * at the end.
+					 */
+					offset = partition_range_bsearch(key, boundinfo, lower,
+													 &equal);
+
+					if (boundinfo->indexes[offset + 1] < 0)
+					{
+						/*
+						 * Check that the new partition will fit in the gap.
+						 * For it to fit, the new upper bound must be less
+						 * than or equal to the lower bound of the next
+						 * partition, if there is one.
+						 */
+						if (offset + 1 < boundinfo->ndatums)
+						{
+							int32		cmpval;
+							Datum 	   *datums;
+							PartitionRangeDatumKind *kind;
+							bool		is_lower;
+
+							datums = boundinfo->datums[offset + 1];
+							kind = boundinfo->kind[offset + 1];
+							is_lower = (boundinfo->indexes[offset + 1] == -1);
+
+							cmpval = partition_rbound_cmp(key, datums, kind,
+														  is_lower, upper);
+							if (cmpval < 0)
+							{
+								/*
+								 * The new partition overlaps with the
+								 * existing partition between offset + 1 and
+								 * offset + 2.
+								 */
+								overlap = true;
+								with = boundinfo->indexes[offset + 2];
+							}
+						}
+					}
+					else
+					{
+						/*
+						 * The new partition overlaps with the existing
+						 * partition between offset and offset + 1.
+						 */
+						overlap = true;
+						with = boundinfo->indexes[offset + 1];
+					}
+				}
+
+				break;
+			}
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	if (overlap)
+	{
+		Assert(with >= 0);
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" would overlap partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[with])),
+				 parser_errposition(pstate, spec->location)));
+	}
+}
+
+/*
+ * RelationGetPartitionQual
+ *
+ * Returns a list of partition quals
+ */
+List *
+RelationGetPartitionQual(Relation rel)
+{
+	/* Quick exit */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
+	return generate_partition_qual(rel);
+}
+
+/*
+ * get_partition_qual_relid
+ *
+ * Returns an expression tree describing the passed-in relation's partition
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
+ */
+Expr *
+get_partition_qual_relid(Oid relid)
+{
+	Relation	rel = heap_open(relid, AccessShareLock);
+	Expr	   *result = NULL;
+	List	   *and_args;
+
+	/* Do the work only if this relation is a partition. */
+	if (rel->rd_rel->relispartition)
+	{
+		and_args = generate_partition_qual(rel);
+
+		if (and_args == NIL)
+			result = NULL;
+		else if (list_length(and_args) > 1)
+			result = makeBoolExpr(AND_EXPR, and_args, -1);
+		else
+			result = linitial(and_args);
+	}
+
+	/* Keep the lock. */
+	heap_close(rel, NoLock);
+
+	return result;
+}
+
+/*
+ * Checks if any of the 'attnums' is a partition key attribute for rel
+ *
+ * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
+ * partition key expression.  It's possible for a column to be both used
+ * directly and as part of an expression; if that happens, *used_in_expr may
+ * end up as either true or false.  That's OK for current uses of this
+ * function, because *used_in_expr is only used to tailor the error message
+ * text.
+ */
+bool
+has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr)
+{
+	PartitionKey key;
+	int			partnatts;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	int			i;
+
+	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		return false;
+
+	key = RelationGetPartitionKey(rel);
+	partnatts = get_partition_natts(key);
+	partexprs = get_partition_exprs(key);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+		{
+			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
+							  attnums))
+			{
+				if (used_in_expr)
+					*used_in_expr = false;
+				return true;
+			}
+		}
+		else
+		{
+			/* Arbitrary expression */
+			Node	   *expr = (Node *) lfirst(partexprs_item);
+			Bitmapset  *expr_attrs = NULL;
+
+			/* Find all attributes referenced */
+			pull_varattnos(expr, 1, &expr_attrs);
+			partexprs_item = lnext(partexprs_item);
+
+			if (bms_overlap(attnums, expr_attrs))
+			{
+				if (used_in_expr)
+					*used_in_expr = true;
+				return true;
+			}
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple
+ *		Finds partition of relation which accepts the partition key specified
+ *		in values and isnull
+ *
+ * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
+ * found or -1 if none found.
+ */
+int
+get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
+{
+	int			bound_offset;
+	int			part_index = -1;
+	PartitionKey key = RelationGetPartitionKey(relation);
+	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
+
+	/* Route as appropriate based on partitioning strategy. */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundInfo boundinfo = partdesc->boundinfo;
+				int			greatest_modulus = get_greatest_modulus(boundinfo);
+				uint64		rowHash = compute_hash_value(key, values, isnull);
+
+				part_index = boundinfo->indexes[rowHash % greatest_modulus];
+			}
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			if (isnull[0])
+			{
+				if (partition_bound_accepts_nulls(partdesc->boundinfo))
+					part_index = partdesc->boundinfo->null_index;
+			}
+			else
+			{
+				bool		equal = false;
+
+				bound_offset = partition_list_bsearch(key,
+													  partdesc->boundinfo,
+													  values[0], &equal);
+				if (bound_offset >= 0 && equal)
+					part_index = partdesc->boundinfo->indexes[bound_offset];
+			}
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				bool		equal = false,
+							range_partkey_has_null = false;
+				int			i;
+
+				/*
+				 * No range includes NULL, so this will be accepted by the
+				 * default partition if there is one, and otherwise rejected.
+				 */
+				for (i = 0; i < key->partnatts; i++)
+				{
+					if (isnull[i])
+					{
+						range_partkey_has_null = true;
+						break;
+					}
+				}
+
+				if (!range_partkey_has_null)
+				{
+					bound_offset = partition_range_datum_bsearch(key,
+														partdesc->boundinfo,
+														key->partnatts,
+														values,
+														&equal);
+					/*
+					 * The bound at bound_offset is less than or equal to the
+					 * tuple value, so the bound at offset+1 is the upper
+					 * bound of the partition we're looking for, if there
+					 * actually exists one.
+					 */
+					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
+				}
+			}
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/*
+	 * part_index < 0 means we failed to find a partition of this parent. Use
+	 * the default partition, if there is one.
+	 */
+	if (part_index < 0)
+		part_index = partdesc->boundinfo->default_index;
+
+	return part_index;
+}
+
+/*
+ * get_greatest_modulus
+ *
+ * Returns the greatest modulus of the hash partition bound. The greatest
+ * modulus will be at the end of the datums array because hash partitions are
+ * arranged in the ascending order of their modulus and remainders.
+ */
+int
+get_greatest_modulus(PartitionBoundInfo bound)
+{
+	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
+	Assert(bound->datums && bound->ndatums > 0);
+	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
+
+	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+uint64
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+	int			i;
+	int			nkeys = key->partnatts;
+	uint64		rowHash = 0;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+	for (i = 0; i < nkeys; i++)
+	{
+		if (!isnull[i])
+		{
+			Datum		hash;
+
+			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
+
+			/*
+			 * Compute hash for each datum value by calling respective
+			 * datatype-specific hash functions of each partition key
+			 * attribute.
+			 */
+			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+	if (partdesc && partdesc->boundinfo &&
+		partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
+
+/* Module-local functions. */
+
+/*
+ * generate_partition_qual
+ *
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
+ *
+ * Result expression tree is stored CacheMemoryContext to ensure it survives
+ * as long as the relcache entry. But we should be running in a less long-lived
+ * working context. To avoid leaking cache memory if this routine fails partway
+ * through, we build in working memory and then copy the completed structure
+ * into cache memory.
+ */
+static List *
+generate_partition_qual(Relation rel)
+{
+	HeapTuple	tuple;
+	MemoryContext oldcxt;
+	Datum		boundDatum;
+	bool		isnull;
+	PartitionBoundSpec *bound;
+	List	   *my_qual = NIL,
+			   *result = NIL;
+	Relation	parent;
+	bool		found_whole_row;
+
+	/* Guard against stack overflow due to overly deep partition tree */
+	check_stack_depth();
+
+	/* Quick copy */
+	if (rel->rd_partcheck != NIL)
+		return copyObject(rel->rd_partcheck);
+
+	/* Grab at least an AccessShareLock on the parent table */
+	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
+					   AccessShareLock);
+
+	/* Get pg_class.relpartbound */
+	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	boundDatum = SysCacheGetAttr(RELOID, tuple,
+								 Anum_pg_class_relpartbound,
+								 &isnull);
+	if (isnull)					/* should not happen */
+		elog(ERROR, "relation \"%s\" has relpartbound = null",
+			 RelationGetRelationName(rel));
+	bound = castNode(PartitionBoundSpec,
+					 stringToNode(TextDatumGetCString(boundDatum)));
+	ReleaseSysCache(tuple);
+
+	my_qual = get_qual_from_partbound(rel, parent, bound);
+
+	/* 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;
+
+	/*
+	 * Change Vars to have partition's attnos instead of the parent's. We do
+	 * this after we concatenate the parent's quals, because we want every Var
+	 * in it to bear this relation's attnos. It's safe to assume varno = 1
+	 * here.
+	 */
+	result = map_partition_varattnos(result, 1, rel, parent,
+									 &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");
+
+	/* Save a copy in the relcache */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	rel->rd_partcheck = copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* Keep the parent locked until commit */
+	heap_close(parent, NoLock);
+
+	return result;
+}
+
+/*
+ * partition_hbound_cmp
+ *
+ * Compares modulus first, then remainder if modulus are equal.
+ */
+static int32
+partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
+{
+	if (modulus1 < modulus2)
+		return -1;
+	if (modulus1 > modulus2)
+		return 1;
+	if (modulus1 == modulus2 && remainder1 != remainder2)
+		return (remainder1 > remainder2) ? 1 : -1;
+	return 0;
+}
+
+/*
+ * qsort_partition_hbound_cmp
+ *
+ * We sort hash bounds by modulus, then by remainder.
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+
+	return partition_hbound_cmp(h1->modulus, h1->remainder,
+								h2->modulus, h2->remainder);
+}
+
+/*
+ * qsort_partition_list_value_cmp
+ *
+ * Compare two list partition bound datums
+ */
+static int32
+qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+{
+	Datum		val1 = (*(const PartitionListValue **) a)->value,
+				val2 = (*(const PartitionListValue **) b)->value;
+	PartitionKey key = (PartitionKey) arg;
+
+	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+										   key->partcollation[0],
+										   val1, val2));
+}
+
+/*
+ * make_one_range_bound
+ *
+ * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
+ * and a flag telling whether the bound is lower or not.  Made into a function
+ * because there are multiple sites that want to use this facility.
+ */
+static PartitionRangeBound *
+make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
+{
+	PartitionRangeBound *bound;
+	ListCell   *lc;
+	int			i;
+
+	Assert(datums != NIL);
+
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
+	bound->index = index;
+	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
+	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
+													  sizeof(PartitionRangeDatumKind));
+	bound->lower = lower;
+
+	i = 0;
+	foreach(lc, datums)
+	{
+		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+
+		/* What's contained in this range datum? */
+		bound->kind[i] = datum->kind;
+
+		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		{
+			Const	   *val = castNode(Const, datum->value);
+
+			if (val->constisnull)
+				elog(ERROR, "invalid range bound datum");
+			bound->datums[i] = val->constvalue;
+		}
+
+		i++;
+	}
+
+	return bound;
+}
+
+/*
+ * partition_rbound_cmp
+ *
+ * Return for two range bounds whether the 1st one (specified in datums1,
+ * kind1, and lower1) is <, =, or > the bound specified in *b2.
+ *
+ * Note that if the values of the two range bounds compare equal, then we take
+ * into account whether they are upper or lower bounds, and an upper bound is
+ * considered to be smaller than a lower bound. This is important to the way
+ * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
+ * structure, which only stores the upper bound of a common boundary between
+ * two contiguous partitions.
+ */
+static int32
+partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2)
+{
+	int32		cmpval = 0;		/* placate compiler */
+	int			i;
+	Datum	   *datums2 = b2->datums;
+	PartitionRangeDatumKind *kind2 = b2->kind;
+	bool		lower2 = b2->lower;
+
+	for (i = 0; i < key->partnatts; i++)
+	{
+		/*
+		 * First, handle cases where the column is unbounded, which should not
+		 * invoke the comparison procedure, and should not consider any later
+		 * columns. Note that the PartitionRangeDatumKind enum elements
+		 * compare the same way as the values they represent.
+		 */
+		if (kind1[i] < kind2[i])
+			return -1;
+		else if (kind1[i] > kind2[i])
+			return 1;
+		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+
+			/*
+			 * The column bounds are both MINVALUE or both MAXVALUE. No later
+			 * columns should be considered, but we still need to compare
+			 * whether they are upper or lower bounds.
+			 */
+			break;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 datums1[i],
+												 datums2[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	/*
+	 * If the comparison is anything other than equal, we're done. If they
+	 * compare equal though, we still have to consider whether the boundaries
+	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
+	 * two.
+	 */
+	if (cmpval == 0 && lower1 != lower2)
+		cmpval = lower1 ? 1 : -1;
+
+	return cmpval;
+}
+
+/* Used when sorting range bounds across all range partitions */
+static int32
+qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
+{
+	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
+	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
+	PartitionKey key = (PartitionKey) arg;
+
+	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+}
+
+/*
+ * partition_list_bsearch
+ *		Returns the index of the greatest bound datum that is less than equal
+ * 		to the given value or -1 if all of the bound datums are greater
+ *
+ * *is_equal is set to true if the bound datum at the returned index is equal
+ * to the input value.
+ */
+static int
+partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+												 key->partcollation[0],
+												 boundinfo->datums[mid][0],
+												 value));
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_rbound_datum_cmp
+ *
+ * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
+ * is <, =, or > partition key of tuple (tuple_datums)
+ */
+static int32
+partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums)
+{
+	int			i;
+	int32		cmpval = -1;
+
+	for (i = 0; i < n_tuple_datums; i++)
+	{
+		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
+			return -1;
+		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
+			return 1;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 rb_datums[i],
+												 tuple_datums[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	return cmpval;
+}
+
+/*
+ * partition_range_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given range bound or -1 if all of the range bounds are
+ *		greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input range bound
+ */
+static int
+partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_cmp(key,
+									  boundinfo->datums[mid],
+									  boundinfo->kind[mid],
+									  (boundinfo->indexes[mid] == -1),
+									  probe);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_range_datum_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given tuple or -1 if all of the range bounds are greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input tuple.
+ */
+static int
+partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_datum_cmp(key,
+											boundinfo->datums[mid],
+											boundinfo->kind[mid],
+											values,
+											nvalues);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_hash_bsearch
+ *		Returns the index of the greatest (modulus, remainder) pair that is
+ *		less than or equal to the given (modulus, remainder) pair or -1 if
+ *		all of them are greater
+ */
+static int
+partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval,
+					bound_modulus,
+					bound_remainder;
+
+		mid = (lo + hi + 1) / 2;
+		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
+		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
+		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
+									  modulus, remainder);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+
+			if (cmpval == 0)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * get_partition_bound_num_indexes
+ *
+ * Returns the number of the entries in the partition bound indexes array.
+ */
+static int
+get_partition_bound_num_indexes(PartitionBoundInfo bound)
+{
+	int			num_indexes;
+
+	Assert(bound);
+
+	switch (bound->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * The number of the entries in the indexes array is same as the
+			 * greatest modulus.
+			 */
+			num_indexes = get_greatest_modulus(bound);
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			num_indexes = bound->ndatums;
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			/* Range partitioned table has an extra index. */
+			num_indexes = bound->ndatums + 1;
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) bound->strategy);
+	}
+
+	return num_indexes;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 1ebf9c4ed2..32fba90be7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -42,7 +42,6 @@
 #include "catalog/index.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_amproc.h"
 #include "catalog/pg_attrdef.h"
@@ -81,6 +80,7 @@
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partcache.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -261,7 +261,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
 static Relation AllocateRelationDesc(Form_pg_class relp);
 static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
 static void RelationBuildTupleDesc(Relation relation);
-static void RelationBuildPartitionKey(Relation relation);
 static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
 static void RelationInitPhysicalAddr(Relation relation);
 static void load_critical_index(Oid indexoid, Oid heapoid);
@@ -809,209 +808,6 @@ RelationBuildRuleLock(Relation relation)
 }
 
 /*
- * RelationBuildPartitionKey
- *		Build and attach to relcache partition key data of relation
- *
- * Partitioning key data is a complex structure; to avoid complicated logic to
- * free individual elements whenever the relcache entry is flushed, we give it
- * its own memory context, child of CacheMemoryContext, which can easily be
- * deleted on its own.  To avoid leaking memory in that context in case of an
- * error partway through this function, the context is initially created as a
- * child of CurTransactionContext and only re-parented to CacheMemoryContext
- * at the end, when no further errors are possible.  Also, we don't make this
- * context the current context except in very brief code sections, out of fear
- * that some of our callees allocate memory on their own which would be leaked
- * permanently.
- */
-static void
-RelationBuildPartitionKey(Relation relation)
-{
-	Form_pg_partitioned_table form;
-	HeapTuple	tuple;
-	bool		isnull;
-	int			i;
-	PartitionKey key;
-	AttrNumber *attrs;
-	oidvector  *opclass;
-	oidvector  *collation;
-	ListCell   *partexprs_item;
-	Datum		datum;
-	MemoryContext partkeycxt,
-				oldcxt;
-	int16		procnum;
-
-	tuple = SearchSysCache1(PARTRELID,
-							ObjectIdGetDatum(RelationGetRelid(relation)));
-
-	/*
-	 * The following happens when we have created our pg_class entry but not
-	 * the pg_partitioned_table entry yet.
-	 */
-	if (!HeapTupleIsValid(tuple))
-		return;
-
-	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
-											   RelationGetRelationName(relation),
-											   MEMCONTEXT_COPY_NAME,
-											   ALLOCSET_SMALL_SIZES);
-
-	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
-												sizeof(PartitionKeyData));
-
-	/* Fixed-length attributes */
-	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	key->strategy = form->partstrat;
-	key->partnatts = form->partnatts;
-
-	/*
-	 * We can rely on the first variable-length attribute being mapped to the
-	 * relevant field of the catalog's C struct, because all previous
-	 * attributes are non-nullable and fixed-length.
-	 */
-	attrs = form->partattrs.values;
-
-	/* But use the hard way to retrieve further variable-length attributes */
-	/* Operator class */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partclass, &isnull);
-	Assert(!isnull);
-	opclass = (oidvector *) DatumGetPointer(datum);
-
-	/* Collation */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partcollation, &isnull);
-	Assert(!isnull);
-	collation = (oidvector *) DatumGetPointer(datum);
-
-	/* Expressions */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partexprs, &isnull);
-	if (!isnull)
-	{
-		char	   *exprString;
-		Node	   *expr;
-
-		exprString = TextDatumGetCString(datum);
-		expr = stringToNode(exprString);
-		pfree(exprString);
-
-		/*
-		 * Run the expressions through const-simplification since the planner
-		 * will be comparing them to similarly-processed qual clause operands,
-		 * and may fail to detect valid matches without this step; fix
-		 * opfuncids while at it.  We don't need to bother with
-		 * canonicalize_qual() though, because partition expressions are not
-		 * full-fledged qualification clauses.
-		 */
-		expr = eval_const_expressions(NULL, expr);
-		fix_opfuncids(expr);
-
-		oldcxt = MemoryContextSwitchTo(partkeycxt);
-		key->partexprs = (List *) copyObject(expr);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
-	oldcxt = MemoryContextSwitchTo(partkeycxt);
-	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
-	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
-
-	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-
-	/* Gather type and collation info as well */
-	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
-	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
-	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
-	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
-	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	MemoryContextSwitchTo(oldcxt);
-
-	/* determine support function number to search for */
-	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
-		HASHEXTENDED_PROC : BTORDER_PROC;
-
-	/* Copy partattrs and fill other per-attribute info */
-	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		AttrNumber	attno = key->partattrs[i];
-		HeapTuple	opclasstup;
-		Form_pg_opclass opclassform;
-		Oid			funcid;
-
-		/* Collect opfamily information */
-		opclasstup = SearchSysCache1(CLAOID,
-									 ObjectIdGetDatum(opclass->values[i]));
-		if (!HeapTupleIsValid(opclasstup))
-			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
-
-		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
-		key->partopfamily[i] = opclassform->opcfamily;
-		key->partopcintype[i] = opclassform->opcintype;
-
-		/* Get a support function for the specified opfamily and datatypes */
-		funcid = get_opfamily_proc(opclassform->opcfamily,
-								   opclassform->opcintype,
-								   opclassform->opcintype,
-								   procnum);
-		if (!OidIsValid(funcid))
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
-							NameStr(opclassform->opcname),
-							(key->strategy == PARTITION_STRATEGY_HASH) ?
-							"hash" : "btree",
-							procnum,
-							format_type_be(opclassform->opcintype))));
-
-		fmgr_info(funcid, &key->partsupfunc[i]);
-
-		/* Collation */
-		key->partcollation[i] = collation->values[i];
-
-		/* Collect type information */
-		if (attno != 0)
-		{
-			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
-
-			key->parttypid[i] = att->atttypid;
-			key->parttypmod[i] = att->atttypmod;
-			key->parttypcoll[i] = att->attcollation;
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			key->parttypid[i] = exprType(lfirst(partexprs_item));
-			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
-			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
-
-			partexprs_item = lnext(partexprs_item);
-		}
-		get_typlenbyvalalign(key->parttypid[i],
-							 &key->parttyplen[i],
-							 &key->parttypbyval[i],
-							 &key->parttypalign[i]);
-
-		ReleaseSysCache(opclasstup);
-	}
-
-	ReleaseSysCache(tuple);
-
-	/*
-	 * Success --- reparent our context and make the relcache point to the
-	 * newly constructed key
-	 */
-	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
-	relation->rd_partkeycxt = partkeycxt;
-	relation->rd_partkey = key;
-}
-
-/*
  *		equalRuleLocks
  *
  *		Determine whether two RuleLocks are equivalent
diff --git a/src/backend/utils/misc/pg_controldata.c b/src/backend/utils/misc/pg_controldata.c
index 8ab7d1337f..87b749a9c4 100644
--- a/src/backend/utils/misc/pg_controldata.c
+++ b/src/backend/utils/misc/pg_controldata.c
@@ -15,6 +15,7 @@
 
 #include "postgres.h"
 
+#include "access/xlog.h"
 #include "funcapi.h"
 #include "miscadmin.h"
 #include "access/htup_details.h"
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2faf0ca26e..59a3fe875b 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -14,63 +14,20 @@
 #define PARTITION_H
 
 #include "fmgr.h"
-#include "executor/tuptable.h"
-#include "nodes/execnodes.h"
-#include "parser/parse_node.h"
-#include "utils/rel.h"
+#include "nodes/parsenodes.h"
+#include "utils/relcache.h"
 
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-/*
- * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
- * associated with partitioned tables as part of its partition descriptor.
- *
- * The internal structure is opaque outside partition.c.
- */
-typedef struct PartitionBoundInfoData *PartitionBoundInfo;
-
-/*
- * Information about partitions of a partitioned table.
- */
-typedef struct PartitionDescData
-{
-	int			nparts;			/* Number of partitions */
-	Oid		   *oids;			/* OIDs of partitions */
-	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
-} PartitionDescData;
-
-typedef struct PartitionDescData *PartitionDesc;
-
-extern void RelationBuildPartitionDesc(Relation relation);
-extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
-					   bool *parttypbyval, PartitionBoundInfo b1,
-					   PartitionBoundInfo b2);
-extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key);
-
-extern void check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec);
 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 fromrel_varno,
-						Relation to_rel, Relation from_rel,
-						bool *found_whole_row);
-extern List *RelationGetPartitionQual(Relation rel);
-extern Expr *get_partition_qual_relid(Oid relid);
-extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr);
 
-extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
 extern Oid	get_default_partition_oid(Oid parentId);
 extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
-extern int get_partition_for_tuple(Relation relation, Datum *values,
-						bool *isnull);
-
 #endif							/* PARTITION_H */
diff --git a/src/include/commands/tablecmds.h b/src/include/commands/tablecmds.h
index 06e5180a30..e786df6f81 100644
--- a/src/include/commands/tablecmds.h
+++ b/src/include/commands/tablecmds.h
@@ -18,7 +18,6 @@
 #include "catalog/dependency.h"
 #include "catalog/objectaddress.h"
 #include "nodes/parsenodes.h"
-#include "catalog/partition.h"
 #include "storage/lock.h"
 #include "utils/relcache.h"
 
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 3df9c498bb..c53dfcc265 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -13,10 +13,10 @@
 #ifndef EXECPARTITION_H
 #define EXECPARTITION_H
 
-#include "catalog/partition.h"
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "utils/partcache.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 1d824eff36..75914028f5 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -14,7 +14,6 @@
 #ifndef EXECUTOR_H
 #define EXECUTOR_H
 
-#include "catalog/partition.h"
 #include "executor/execdesc.h"
 #include "nodes/parsenodes.h"
 #include "utils/memutils.h"
diff --git a/src/include/rewrite/rewriteManip.h b/src/include/rewrite/rewriteManip.h
index f0299bc703..7ca8a81931 100644
--- a/src/include/rewrite/rewriteManip.h
+++ b/src/include/rewrite/rewriteManip.h
@@ -15,6 +15,7 @@
 #define REWRITEMANIP_H
 
 #include "nodes/parsenodes.h"
+#include "utils/rel.h"
 
 
 typedef struct replace_rte_variables_context replace_rte_variables_context;
@@ -73,6 +74,9 @@ extern Node *map_variable_attnos(Node *node,
 					int target_varno, int sublevels_up,
 					const AttrNumber *attno_map, int map_length,
 					Oid to_rowtype, bool *found_whole_row);
+extern List *map_partition_varattnos(List *expr, int fromrel_varno,
+						Relation to_rel, Relation from_rel,
+						bool *found_whole_row);
 
 extern Node *ReplaceVarsFromTargetList(Node *node,
 						  int target_varno, int sublevels_up,
diff --git a/src/include/utils/partcache.h b/src/include/utils/partcache.h
new file mode 100644
index 0000000000..0f9c1cdfd5
--- /dev/null
+++ b/src/include/utils/partcache.h
@@ -0,0 +1,188 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.h
+ *		Header file for partitioning related cached data structures and
+ *		manipulation functions
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partcache.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTCACHE_H
+#define PARTCACHE_H
+
+#include "fmgr.h"
+#include "utils/lsyscache.h"
+#include "utils/relcache.h"
+
+/*
+ * Information about the partition key of a relation
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partitioning strategy */
+	int16		partnatts;		/* number of columns in the partition key */
+	AttrNumber *partattrs;		/* attribute numbers of columns in the
+								 * partition key */
+	List	   *partexprs;		/* list of expressions in the partitioning
+								 * key, or NIL */
+
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+
+	/* Partitioning collation per attribute */
+	Oid		   *partcollation;
+
+	/* Type information per attribute */
+	Oid		   *parttypid;
+	int32	   *parttypmod;
+	int16	   *parttyplen;
+	bool	   *parttypbyval;
+	char	   *parttypalign;
+	Oid		   *parttypcoll;
+}			PartitionKeyData;
+
+typedef struct PartitionKeyData *PartitionKey;
+
+typedef struct PartitionBoundInfoData *PartitionBoundInfo;
+
+/*
+ * Information about partitions of a partitioned table.
+ */
+typedef struct PartitionDescData
+{
+	int			nparts;			/* Number of partitions */
+	Oid		   *oids;			/* OIDs of partitions */
+	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
+} PartitionDescData;
+
+typedef struct PartitionDescData *PartitionDesc;
+
+/*
+ * Information about bounds of a partitioned relation
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+	char		strategy;		/* hash, list or range? */
+	int			ndatums;		/* Length of the datums following array */
+	Datum	  **datums;
+	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+									 * NULL for hash and list partitioned
+									 * tables */
+	int		   *indexes;		/* Partition indexes */
+	int			null_index;		/* Index of the null-accepting partition; -1
+								 * if there isn't one */
+	int			default_index;	/* Index of the default partition; -1 if there
+								 * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * PartitionKey inquiry functions
+ */
+static inline int
+get_partition_strategy(PartitionKey key)
+{
+	return key->strategy;
+}
+
+static inline int
+get_partition_natts(PartitionKey key)
+{
+	return key->partnatts;
+}
+
+static inline List *
+get_partition_exprs(PartitionKey key)
+{
+	return key->partexprs;
+}
+
+/*
+ * PartitionKey inquiry functions - one column
+ */
+static inline int16
+get_partition_col_attnum(PartitionKey key, int col)
+{
+	return key->partattrs[col];
+}
+
+static inline Oid
+get_partition_col_typid(PartitionKey key, int col)
+{
+	return key->parttypid[col];
+}
+
+static inline int32
+get_partition_col_typmod(PartitionKey key, int col)
+{
+	return key->parttypmod[col];
+}
+
+extern void RelationBuildPartitionKey(Relation relation);
+extern void RelationBuildPartitionDesc(Relation relation);
+extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
+					   bool *parttypbyval, PartitionBoundInfo b1,
+					   PartitionBoundInfo b2);
+
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
+
+extern void check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec);
+
+extern List *RelationGetPartitionQual(Relation rel);
+extern Expr *get_partition_qual_relid(Oid relid);
+
+extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr);
+
+extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+
+extern int get_greatest_modulus(PartitionBoundInfo b);
+extern uint64 compute_hash_value(PartitionKey key, Datum *values,
+						bool *isnull);
+
+/* For tuple routing */
+extern int get_partition_for_tuple(Relation relation, Datum *values,
+						bool *isnull);
+
+#endif							/* PARTCACHE_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index aa8add544a..b531ef0121 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -24,6 +24,7 @@
 #include "rewrite/prs2lock.h"
 #include "storage/block.h"
 #include "storage/relfilenode.h"
+#include "utils/partcache.h"
 #include "utils/relcache.h"
 #include "utils/reltrigger.h"
 
@@ -47,36 +48,6 @@ typedef struct LockInfoData
 typedef LockInfoData *LockInfo;
 
 /*
- * Information about the partition key of a relation
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partitioning strategy */
-	int16		partnatts;		/* number of columns in the partition key */
-	AttrNumber *partattrs;		/* attribute numbers of columns in the
-								 * partition key */
-	List	   *partexprs;		/* list of expressions in the partitioning
-								 * key, or NIL */
-
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-
-	/* Partitioning collation per attribute */
-	Oid		   *partcollation;
-
-	/* Type information per attribute */
-	Oid		   *parttypid;
-	int32	   *parttypmod;
-	int16	   *parttyplen;
-	bool	   *parttypbyval;
-	char	   *parttypalign;
-	Oid		   *parttypcoll;
-}			PartitionKeyData;
-
-typedef struct PartitionKeyData *PartitionKey;
-
-/*
  * Here are the contents of a relation cache entry.
  */
 
@@ -593,48 +564,6 @@ typedef struct ViewOptions
 #define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
 
 /*
- * PartitionKey inquiry functions
- */
-static inline int
-get_partition_strategy(PartitionKey key)
-{
-	return key->strategy;
-}
-
-static inline int
-get_partition_natts(PartitionKey key)
-{
-	return key->partnatts;
-}
-
-static inline List *
-get_partition_exprs(PartitionKey key)
-{
-	return key->partexprs;
-}
-
-/*
- * PartitionKey inquiry functions - one column
- */
-static inline int16
-get_partition_col_attnum(PartitionKey key, int col)
-{
-	return key->partattrs[col];
-}
-
-static inline Oid
-get_partition_col_typid(PartitionKey key, int col)
-{
-	return key->parttypid[col];
-}
-
-static inline int32
-get_partition_col_typmod(PartitionKey key, int col)
-{
-	return key->parttypmod[col];
-}
-
-/*
  * RelationGetPartitionDesc
  *		Returns partition descriptor for a relation.
  */
-- 
2.11.0

#8Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#7)
Re: reorganizing partitioning code

Added to CF here: https://commitfest.postgresql.org/17/1520/

Thanks,
Amit

#9Kyotaro HORIGUCHI
horiguchi.kyotaro@lab.ntt.co.jp
In reply to: Amit Langote (#8)
Re: reorganizing partitioning code

Hello. I'd like to make a humble comment.

At Thu, 15 Feb 2018 19:31:47 +0900, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote in <8906c861-ea47-caee-c860-eff8d7e1dbc0@lab.ntt.co.jp>

Added to CF here: https://commitfest.postgresql.org/17/1520/

The reorganization adds/removes header files to/from .[ch]
files. That can easily leave useless includes and they're hardly
noticed. Following files touched in this patch have such useless
includes after this patch.

On the other hand, naive decision of this kind of cleanup can
lead to curruption. [1]/messages/by-id/6748.1518711125@sss.pgh.pa.us So, I don't insisit that the all of the
following *should* amended, especially for rel.h.

[1]: /messages/by-id/6748.1518711125@sss.pgh.pa.us

==== nodeModifyTable.c:

+#include "rewrite/rewriteManip.h"

rewriteManip.h is changed to include rel.h so rel.h is no longer
need to be included in the file. (It is also included in lmgr.h
so it was needless since before this patch, though.)

==== hba.c:

+#include "catalog/objectaddress.h"

objectaddress.h includes acl.h so acl.h is no longer useful.

==== joinrels.c:

+#include "utils/partcache.h"

partcache.h includes lsyscache.h.

==== prepunion.c:

+#include "utils/partcache.h"

partcache.h includes lsyscache.h and partcache.h is included in
rel.h. So partcache.h and lsyscache.h are not required.

==== utility.c:

+#include "utils/rel.h"

rel.h includes xlog.h (and xlog.h includes fd.h). The last two
are removable.

==== partcache.c:
parsenodes.h is included from some other files.
rel.h is included from rewriteManip.h.
partcache.h is included from rel.h
As the result, parsenodes.h, rel.h, partcache.h are not required.

==== relcache.c:

+#include "utils/partcache.h"

lsyscache.h is included by partcache.h.

==== rel.h:

+#include "utils/partcache.h"

partcache.h includes fmgr.h and relcache.h so the last two are
no longer useful.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#10Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Kyotaro HORIGUCHI (#9)
1 attachment(s)
Re: reorganizing partitioning code

Horiguchi-san,

On 2018/02/16 14:07, Kyotaro HORIGUCHI wrote:

Hello. I'd like to make a humble comment.

Thanks a lot for taking a look.

The reorganization adds/removes header files to/from .[ch]
files. That can easily leave useless includes and they're hardly
noticed. Following files touched in this patch have such useless
includes after this patch.

Yes, I agree.

On the other hand, naive decision of this kind of cleanup can
lead to curruption. [1] So, I don't insisit that the all of the
following *should* amended, especially for rel.h.

[1] /messages/by-id/6748.1518711125@sss.pgh.pa.us

I was initially trying limit this patch's #include churn so that it only
touches .[ch] files that are somehow involved with partitioning, but ended
up with a lot of files touched. To prevent confusion and concerns, I have
updated the patch to keep the churn to minimum as best as I could.

So, while the v3 patch looked like this:

24 files changed, 2373 insertions(+), 2312 deletions(-)
create mode 100644 src/backend/utils/cache/partcache.c
create mode 100644 src/include/utils/partcache.h

v4 looks like this:

15 files changed, 2314 insertions(+), 2263 deletions(-)
create mode 100644 src/backend/utils/cache/partcache.c
create mode 100644 src/include/utils/partcache.h

==== nodeModifyTable.c:

+#include "rewrite/rewriteManip.h"

rewriteManip.h is changed to include rel.h so rel.h is no longer
need to be included in the file. (It is also included in lmgr.h
so it was needless since before this patch, though.)

On second thought, I decided to hold back on moving
map_partition_varattnos() to rewriteManip.c, because its interface is
unlike any other functions in that file, requiring us to include rel.h in
rewriteManip.h. So, I removed this #include from nodeModifyTable.c.

==== hba.c:

+#include "catalog/objectaddress.h"

objectaddress.h includes acl.h so acl.h is no longer useful.

This and a few others were necessitated by removing partition.h from
executor.h. For now, I'm putting partition.h back into executor.h,
although it would be nice to remove it eventually.

==== joinrels.c:

+#include "utils/partcache.h"

partcache.h includes lsyscache.h.

Moved lsyscache.h from partcache.h to partcache.c.

==== prepunion.c:

+#include "utils/partcache.h"

partcache.h includes lsyscache.h and partcache.h is included in
rel.h. So partcache.h and lsyscache.h are not required.

Oops, why did I even include partcache.h in prepunion.c!

Removed.

==== utility.c:

+#include "utils/rel.h"

rel.h includes xlog.h (and xlog.h includes fd.h). The last two
are removable.

I've reverted the change that necessitated this and so this one.

==== partcache.c:
parsenodes.h is included from some other files.
rel.h is included from rewriteManip.h.
partcache.h is included from rel.h
As the result, parsenodes.h, rel.h, partcache.h are not required.

Removed.

==== relcache.c:

+#include "utils/partcache.h"

lsyscache.h is included by partcache.h.

lsyscache.h was moved from partcache.h per above, so keeping here.

==== rel.h:

+#include "utils/partcache.h"

partcache.h includes fmgr.h and relcache.h so the last two are
no longer useful.

Removed both.

Attached updated version.

Thanks,
Amit

Attachments:

v4-0001-Reorganize-partitioning-code.patchtext/plain; charset=UTF-8; name=v4-0001-Reorganize-partitioning-code.patchDownload
From 7e59c27070ae358ed37fbd1d0e5b37faf286ac3d Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Tue, 13 Feb 2018 15:59:30 +0900
Subject: [PATCH v4] Reorganize partitioning code

---
 src/backend/catalog/partition.c        | 1934 +----------------------------
 src/backend/executor/execMain.c        |    1 -
 src/backend/executor/execPartition.c   |    1 +
 src/backend/optimizer/path/joinrels.c  |    2 +-
 src/backend/optimizer/prep/prepunion.c |    1 -
 src/backend/optimizer/util/plancat.c   |    1 -
 src/backend/optimizer/util/relnode.c   |    2 +-
 src/backend/utils/adt/ruleutils.c      |    1 -
 src/backend/utils/cache/Makefile       |    6 +-
 src/backend/utils/cache/partcache.c    | 2116 ++++++++++++++++++++++++++++++++
 src/backend/utils/cache/relcache.c     |  206 +---
 src/include/catalog/partition.h        |   42 +-
 src/include/executor/execPartition.h   |    2 +-
 src/include/utils/partcache.h          |  187 +++
 src/include/utils/rel.h                |   75 +-
 15 files changed, 2314 insertions(+), 2263 deletions(-)
 create mode 100644 src/backend/utils/cache/partcache.c
 create mode 100644 src/include/utils/partcache.h

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 31c80c7f1a..01198bea9e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,11 +15,7 @@
 
 #include "postgres.h"
 
-#include "access/hash.h"
-#include "access/heapam.h"
 #include "access/htup_details.h"
-#include "access/nbtree.h"
-#include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
@@ -35,7 +31,6 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
-#include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
 #include "optimizer/prep.h"
@@ -48,102 +43,12 @@
 #include "utils/datum.h"
 #include "utils/fmgroids.h"
 #include "utils/hashutils.h"
-#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
-#include "utils/ruleutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-	char		strategy;		/* hash, list or range? */
-	int			ndatums;		/* Length of the datums following array */
-	Datum	  **datums;
-	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-									 * NULL for hash and list partitioned
-									 * tables */
-	int		   *indexes;		/* Partition indexes */
-	int			null_index;		/* Index of the null-accepting partition; -1
-								 * if there isn't one */
-	int			default_index;	/* Index of the default partition; -1 if there
-								 * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-	int			modulus;
-	int			remainder;
-	int			index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-	int			index;
-	Datum		value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-	int			index;
-	Datum	   *datums;			/* range bound datums */
-	PartitionRangeDatumKind *kind;	/* the kind of each datum */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
-static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
-static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
-							   void *arg);
-static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
-						   void *arg);
-
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
@@ -159,1069 +64,11 @@ static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 				   bool for_default);
 static List *get_range_nulltest(PartitionKey key);
-static List *generate_partition_qual(Relation rel);
-
-static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
-					 List *datums, bool lower);
-static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
-					 int remainder2);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal);
-static int partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder);
-
-static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int	get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 
 /* SQL-callable function for use in hash partition CHECK constraints */
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 /*
- * RelationBuildPartitionDesc
- *		Form rel's partition descriptor
- *
- * Not flushed from the cache by RelationClearRelation() unless changed because
- * of addition or removal of partition.
- */
-void
-RelationBuildPartitionDesc(Relation rel)
-{
-	List	   *inhoids,
-			   *partoids;
-	Oid		   *oids = NULL;
-	List	   *boundspecs = NIL;
-	ListCell   *cell;
-	int			i,
-				nparts;
-	PartitionKey key = RelationGetPartitionKey(rel);
-	PartitionDesc result;
-	MemoryContext oldcxt;
-
-	int			ndatums = 0;
-	int			default_index = -1;
-
-	/* Hash partitioning specific */
-	PartitionHashBound **hbounds = NULL;
-
-	/* List partitioning specific */
-	PartitionListValue **all_values = NULL;
-	int			null_index = -1;
-
-	/* Range partitioning specific */
-	PartitionRangeBound **rbounds = NULL;
-
-	/*
-	 * The following could happen in situations where rel has a pg_class entry
-	 * but not the pg_partitioned_table entry yet.
-	 */
-	if (key == NULL)
-		return;
-
-	/* Get partition oids from pg_inherits */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
-
-	/* Collect bound spec nodes in a list */
-	i = 0;
-	partoids = NIL;
-	foreach(cell, inhoids)
-	{
-		Oid			inhrelid = lfirst_oid(cell);
-		HeapTuple	tuple;
-		Datum		datum;
-		bool		isnull;
-		Node	   *boundspec;
-
-		tuple = SearchSysCache1(RELOID, inhrelid);
-		if (!HeapTupleIsValid(tuple))
-			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
-
-		/*
-		 * It is possible that the pg_class tuple of a partition has not been
-		 * updated yet to set its relpartbound field.  The only case where
-		 * this happens is when we open the parent relation to check using its
-		 * partition descriptor that a new partition's bound does not overlap
-		 * some existing partition.
-		 */
-		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
-		{
-			ReleaseSysCache(tuple);
-			continue;
-		}
-
-		datum = SysCacheGetAttr(RELOID, tuple,
-								Anum_pg_class_relpartbound,
-								&isnull);
-		Assert(!isnull);
-		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
-
-		/*
-		 * Sanity check: If the PartitionBoundSpec says this is the default
-		 * partition, its OID should correspond to whatever's stored in
-		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
-		 */
-		if (castNode(PartitionBoundSpec, boundspec)->is_default)
-		{
-			Oid			partdefid;
-
-			partdefid = get_default_partition_oid(RelationGetRelid(rel));
-			if (partdefid != inhrelid)
-				elog(ERROR, "expected partdefid %u, but got %u",
-					 inhrelid, partdefid);
-		}
-
-		boundspecs = lappend(boundspecs, boundspec);
-		partoids = lappend_oid(partoids, inhrelid);
-		ReleaseSysCache(tuple);
-	}
-
-	nparts = list_length(partoids);
-
-	if (nparts > 0)
-	{
-		oids = (Oid *) palloc(nparts * sizeof(Oid));
-		i = 0;
-		foreach(cell, partoids)
-			oids[i++] = lfirst_oid(cell);
-
-		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_HASH)
-		{
-			ndatums = nparts;
-			hbounds = (PartitionHashBound **)
-				palloc(nparts * sizeof(PartitionHashBound *));
-
-			i = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-
-				if (spec->strategy != PARTITION_STRATEGY_HASH)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				hbounds[i] = (PartitionHashBound *)
-					palloc(sizeof(PartitionHashBound));
-
-				hbounds[i]->modulus = spec->modulus;
-				hbounds[i]->remainder = spec->remainder;
-				hbounds[i]->index = i;
-				i++;
-			}
-
-			/* Sort all the bounds in ascending order */
-			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
-				  qsort_partition_hbound_cmp);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_LIST)
-		{
-			List	   *non_null_values = NIL;
-
-			/*
-			 * Create a unified list of non-null values across all partitions.
-			 */
-			i = 0;
-			null_index = -1;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				ListCell   *c;
-
-				if (spec->strategy != PARTITION_STRATEGY_LIST)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the list of non-null
-				 * datums for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i;
-					i++;
-					continue;
-				}
-
-				foreach(c, spec->listdatums)
-				{
-					Const	   *val = castNode(Const, lfirst(c));
-					PartitionListValue *list_value = NULL;
-
-					if (!val->constisnull)
-					{
-						list_value = (PartitionListValue *)
-							palloc0(sizeof(PartitionListValue));
-						list_value->index = i;
-						list_value->value = val->constvalue;
-					}
-					else
-					{
-						/*
-						 * Never put a null into the values array, flag
-						 * instead for the code further down below where we
-						 * construct the actual relcache struct.
-						 */
-						if (null_index != -1)
-							elog(ERROR, "found null more than once");
-						null_index = i;
-					}
-
-					if (list_value)
-						non_null_values = lappend(non_null_values,
-												  list_value);
-				}
-
-				i++;
-			}
-
-			ndatums = list_length(non_null_values);
-
-			/*
-			 * Collect all list values in one array. Alongside the value, we
-			 * also save the index of partition the value comes from.
-			 */
-			all_values = (PartitionListValue **) palloc(ndatums *
-														sizeof(PartitionListValue *));
-			i = 0;
-			foreach(cell, non_null_values)
-			{
-				PartitionListValue *src = lfirst(cell);
-
-				all_values[i] = (PartitionListValue *)
-					palloc(sizeof(PartitionListValue));
-				all_values[i]->value = src->value;
-				all_values[i]->index = src->index;
-				i++;
-			}
-
-			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
-					  qsort_partition_list_value_cmp, (void *) key);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_RANGE)
-		{
-			int			k;
-			PartitionRangeBound **all_bounds,
-					   *prev;
-
-			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
-														  sizeof(PartitionRangeBound *));
-
-			/*
-			 * Create a unified list of range bounds across all the
-			 * partitions.
-			 */
-			i = ndatums = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				PartitionRangeBound *lower,
-						   *upper;
-
-				if (spec->strategy != PARTITION_STRATEGY_RANGE)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the allbounds array
-				 * for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i++;
-					continue;
-				}
-
-				lower = make_one_range_bound(key, i, spec->lowerdatums,
-											 true);
-				upper = make_one_range_bound(key, i, spec->upperdatums,
-											 false);
-				all_bounds[ndatums++] = lower;
-				all_bounds[ndatums++] = upper;
-				i++;
-			}
-
-			Assert(ndatums == nparts * 2 ||
-				   (default_index != -1 && ndatums == (nparts - 1) * 2));
-
-			/* Sort all the bounds in ascending order */
-			qsort_arg(all_bounds, ndatums,
-					  sizeof(PartitionRangeBound *),
-					  qsort_partition_rbound_cmp,
-					  (void *) key);
-
-			/* Save distinct bounds from all_bounds into rbounds. */
-			rbounds = (PartitionRangeBound **)
-				palloc(ndatums * sizeof(PartitionRangeBound *));
-			k = 0;
-			prev = NULL;
-			for (i = 0; i < ndatums; i++)
-			{
-				PartitionRangeBound *cur = all_bounds[i];
-				bool		is_distinct = false;
-				int			j;
-
-				/* Is the current bound distinct from the previous one? */
-				for (j = 0; j < key->partnatts; j++)
-				{
-					Datum		cmpval;
-
-					if (prev == NULL || cur->kind[j] != prev->kind[j])
-					{
-						is_distinct = true;
-						break;
-					}
-
-					/*
-					 * If the bounds are both MINVALUE or MAXVALUE, stop now
-					 * and treat them as equal, since any values after this
-					 * point must be ignored.
-					 */
-					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
-						break;
-
-					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
-											   key->partcollation[j],
-											   cur->datums[j],
-											   prev->datums[j]);
-					if (DatumGetInt32(cmpval) != 0)
-					{
-						is_distinct = true;
-						break;
-					}
-				}
-
-				/*
-				 * Only if the bound is distinct save it into a temporary
-				 * array i.e. rbounds which is later copied into boundinfo
-				 * datums array.
-				 */
-				if (is_distinct)
-					rbounds[k++] = all_bounds[i];
-
-				prev = cur;
-			}
-
-			/* Update ndatums to hold the count of distinct datums. */
-			ndatums = k;
-		}
-		else
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/* Now build the actual relcache partition descriptor */
-	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
-												  RelationGetRelationName(rel),
-												  MEMCONTEXT_COPY_NAME,
-												  ALLOCSET_DEFAULT_SIZES);
-	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
-
-	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
-	result->nparts = nparts;
-	if (nparts > 0)
-	{
-		PartitionBoundInfo boundinfo;
-		int		   *mapping;
-		int			next_index = 0;
-
-		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
-
-		boundinfo = (PartitionBoundInfoData *)
-			palloc0(sizeof(PartitionBoundInfoData));
-		boundinfo->strategy = key->strategy;
-		boundinfo->default_index = -1;
-		boundinfo->ndatums = ndatums;
-		boundinfo->null_index = -1;
-		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
-
-		/* Initialize mapping array with invalid values */
-		mapping = (int *) palloc(sizeof(int) * nparts);
-		for (i = 0; i < nparts; i++)
-			mapping[i] = -1;
-
-		switch (key->strategy)
-		{
-			case PARTITION_STRATEGY_HASH:
-				{
-					/* Modulus are stored in ascending order */
-					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
-
-					boundinfo->indexes = (int *) palloc(greatest_modulus *
-														sizeof(int));
-
-					for (i = 0; i < greatest_modulus; i++)
-						boundinfo->indexes[i] = -1;
-
-					for (i = 0; i < nparts; i++)
-					{
-						int			modulus = hbounds[i]->modulus;
-						int			remainder = hbounds[i]->remainder;
-
-						boundinfo->datums[i] = (Datum *) palloc(2 *
-																sizeof(Datum));
-						boundinfo->datums[i][0] = Int32GetDatum(modulus);
-						boundinfo->datums[i][1] = Int32GetDatum(remainder);
-
-						while (remainder < greatest_modulus)
-						{
-							/* overlap? */
-							Assert(boundinfo->indexes[remainder] == -1);
-							boundinfo->indexes[remainder] = i;
-							remainder += modulus;
-						}
-
-						mapping[hbounds[i]->index] = i;
-						pfree(hbounds[i]);
-					}
-					pfree(hbounds);
-					break;
-				}
-
-			case PARTITION_STRATEGY_LIST:
-				{
-					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
-
-					/*
-					 * Copy values.  Indexes of individual values are mapped
-					 * to canonical values so that they match for any two list
-					 * partitioned tables with same number of partitions and
-					 * same lists per partition.  One way to canonicalize is
-					 * to assign the index in all_values[] of the smallest
-					 * value of each partition, as the index of all of the
-					 * partition's values.
-					 */
-					for (i = 0; i < ndatums; i++)
-					{
-						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
-						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
-															key->parttypbyval[0],
-															key->parttyplen[0]);
-
-						/* If the old index has no mapping, assign one */
-						if (mapping[all_values[i]->index] == -1)
-							mapping[all_values[i]->index] = next_index++;
-
-						boundinfo->indexes[i] = mapping[all_values[i]->index];
-					}
-
-					/*
-					 * If null-accepting partition has no mapped index yet,
-					 * assign one.  This could happen if such partition
-					 * accepts only null and hence not covered in the above
-					 * loop which only handled non-null values.
-					 */
-					if (null_index != -1)
-					{
-						Assert(null_index >= 0);
-						if (mapping[null_index] == -1)
-							mapping[null_index] = next_index++;
-						boundinfo->null_index = mapping[null_index];
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						/*
-						 * The default partition accepts any value not
-						 * specified in the lists of other partitions, hence
-						 * it should not get mapped index while assigning
-						 * those for non-null datums.
-						 */
-						Assert(default_index >= 0 &&
-							   mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-
-					/* All partition must now have a valid mapping */
-					Assert(next_index == nparts);
-					break;
-				}
-
-			case PARTITION_STRATEGY_RANGE:
-				{
-					boundinfo->kind = (PartitionRangeDatumKind **)
-						palloc(ndatums *
-							   sizeof(PartitionRangeDatumKind *));
-					boundinfo->indexes = (int *) palloc((ndatums + 1) *
-														sizeof(int));
-
-					for (i = 0; i < ndatums; i++)
-					{
-						int			j;
-
-						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
-																sizeof(Datum));
-						boundinfo->kind[i] = (PartitionRangeDatumKind *)
-							palloc(key->partnatts *
-								   sizeof(PartitionRangeDatumKind));
-						for (j = 0; j < key->partnatts; j++)
-						{
-							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
-								boundinfo->datums[i][j] =
-									datumCopy(rbounds[i]->datums[j],
-											  key->parttypbyval[j],
-											  key->parttyplen[j]);
-							boundinfo->kind[i][j] = rbounds[i]->kind[j];
-						}
-
-						/*
-						 * There is no mapping for invalid indexes.
-						 *
-						 * Any lower bounds in the rbounds array have invalid
-						 * indexes assigned, because the values between the
-						 * previous bound (if there is one) and this (lower)
-						 * bound are not part of the range of any existing
-						 * partition.
-						 */
-						if (rbounds[i]->lower)
-							boundinfo->indexes[i] = -1;
-						else
-						{
-							int			orig_index = rbounds[i]->index;
-
-							/* If the old index has no mapping, assign one */
-							if (mapping[orig_index] == -1)
-								mapping[orig_index] = next_index++;
-
-							boundinfo->indexes[i] = mapping[orig_index];
-						}
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						Assert(default_index >= 0 && mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-					boundinfo->indexes[i] = -1;
-					break;
-				}
-
-			default:
-				elog(ERROR, "unexpected partition strategy: %d",
-					 (int) key->strategy);
-		}
-
-		result->boundinfo = boundinfo;
-
-		/*
-		 * Now assign OIDs from the original array into mapped indexes of the
-		 * result array.  Order of OIDs in the former is defined by the
-		 * catalog scan that retrieved them, whereas that in the latter is
-		 * defined by canonicalized representation of the partition bounds.
-		 */
-		for (i = 0; i < nparts; i++)
-			result->oids[mapping[i]] = oids[i];
-		pfree(mapping);
-	}
-
-	MemoryContextSwitchTo(oldcxt);
-	rel->rd_partdesc = result;
-}
-
-/*
- * Are two partition bound collections logically equal?
- *
- * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
- * This is also useful when b1 and b2 are bound collections of two separate
- * relations, respectively, because PartitionBoundInfo is a canonical
- * representation of partition bounds.
- */
-bool
-partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
-					   PartitionBoundInfo b1, PartitionBoundInfo b2)
-{
-	int			i;
-
-	if (b1->strategy != b2->strategy)
-		return false;
-
-	if (b1->ndatums != b2->ndatums)
-		return false;
-
-	if (b1->null_index != b2->null_index)
-		return false;
-
-	if (b1->default_index != b2->default_index)
-		return false;
-
-	if (b1->strategy == PARTITION_STRATEGY_HASH)
-	{
-		int			greatest_modulus = get_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_greatest_modulus(b2))
-			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their modulus
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
-	}
-	else
-	{
-		for (i = 0; i < b1->ndatums; i++)
-		{
-			int			j;
-
-			for (j = 0; j < partnatts; j++)
-			{
-				/* For range partitions, the bounds might not be finite. */
-				if (b1->kind != NULL)
-				{
-					/* The different kinds of bound all differ from each other */
-					if (b1->kind[i][j] != b2->kind[i][j])
-						return false;
-
-					/*
-					 * Non-finite bounds are equal without further
-					 * examination.
-					 */
-					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
-						continue;
-				}
-
-				/*
-				 * Compare the actual values. Note that it would be both
-				 * incorrect and unsafe to invoke the comparison operator
-				 * derived from the partitioning specification here.  It would
-				 * be incorrect because we want the relcache entry to be
-				 * updated for ANY change to the partition bounds, not just
-				 * those that the partitioning operator thinks are
-				 * significant.  It would be unsafe because we might reach
-				 * this code in the context of an aborted transaction, and an
-				 * arbitrary partitioning operator might not be safe in that
-				 * context.  datumIsEqual() should be simple enough to be
-				 * safe.
-				 */
-				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-								  parttypbyval[j], parttyplen[j]))
-					return false;
-			}
-
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-		}
-
-		/* There are ndatums+1 indexes in case of range partitions */
-		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
-			b1->indexes[i] != b2->indexes[i])
-			return false;
-	}
-	return true;
-}
-
-/*
- * Return a copy of given PartitionBoundInfo structure. The data types of bounds
- * are described by given partition key specification.
- */
-extern PartitionBoundInfo
-partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key)
-{
-	PartitionBoundInfo dest;
-	int			i;
-	int			ndatums;
-	int			partnatts;
-	int			num_indexes;
-
-	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
-
-	dest->strategy = src->strategy;
-	ndatums = dest->ndatums = src->ndatums;
-	partnatts = key->partnatts;
-
-	num_indexes = get_partition_bound_num_indexes(src);
-
-	/* List partitioned tables have only a single partition key. */
-	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
-
-	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
-
-	if (src->kind != NULL)
-	{
-		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
-														 sizeof(PartitionRangeDatumKind *));
-		for (i = 0; i < ndatums; i++)
-		{
-			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
-															   sizeof(PartitionRangeDatumKind));
-
-			memcpy(dest->kind[i], src->kind[i],
-				   sizeof(PartitionRangeDatumKind) * key->partnatts);
-		}
-	}
-	else
-		dest->kind = NULL;
-
-	for (i = 0; i < ndatums; i++)
-	{
-		int			j;
-
-		/*
-		 * For a corresponding to hash partition, datums array will have two
-		 * elements - modulus and remainder.
-		 */
-		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
-		int			natts = hash_part ? 2 : partnatts;
-
-		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
-
-		for (j = 0; j < natts; j++)
-		{
-			bool		byval;
-			int			typlen;
-
-			if (hash_part)
-			{
-				typlen = sizeof(int32); /* Always int4 */
-				byval = true;	/* int4 is pass-by-value */
-			}
-			else
-			{
-				byval = key->parttypbyval[j];
-				typlen = key->parttyplen[j];
-			}
-
-			if (dest->kind == NULL ||
-				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
-				dest->datums[i][j] = datumCopy(src->datums[i][j],
-											   byval, typlen);
-		}
-	}
-
-	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
-	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
-
-	dest->null_index = src->null_index;
-	dest->default_index = src->default_index;
-
-	return dest;
-}
-
-/*
- * check_new_partition_bound
- *
- * Checks if the new partition's bound overlaps any of the existing partitions
- * of parent.  Also performs additional checks as necessary per strategy.
- */
-void
-check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	PartitionBoundInfo boundinfo = partdesc->boundinfo;
-	ParseState *pstate = make_parsestate(NULL);
-	int			with = -1;
-	bool		overlap = false;
-
-	if (spec->is_default)
-	{
-		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
-			return;
-
-		/* Default partition already exists, error out. */
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
-				 parser_errposition(pstate, spec->location)));
-	}
-
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					Datum	  **datums = boundinfo->datums;
-					int			ndatums = boundinfo->ndatums;
-					int			greatest_modulus;
-					int			remainder;
-					int			offset;
-					bool		valid_modulus = true;
-					int			prev_modulus,	/* Previous largest modulus */
-								next_modulus;	/* Next largest modulus */
-
-					/*
-					 * Check rule that every modulus must be a factor of the
-					 * next larger modulus.  For example, if you have a bunch
-					 * of partitions that all have modulus 5, you can add a
-					 * new partition with modulus 10 or a new partition with
-					 * modulus 15, but you cannot add both a partition with
-					 * modulus 10 and a partition with modulus 15, because 10
-					 * is not a factor of 15.
-					 *
-					 * Get the greatest (modulus, remainder) pair contained in
-					 * boundinfo->datums that is less than or equal to the
-					 * (spec->modulus, spec->remainder) pair.
-					 */
-					offset = partition_hash_bsearch(key, boundinfo,
-													spec->modulus,
-													spec->remainder);
-					if (offset < 0)
-					{
-						next_modulus = DatumGetInt32(datums[0][0]);
-						valid_modulus = (next_modulus % spec->modulus) == 0;
-					}
-					else
-					{
-						prev_modulus = DatumGetInt32(datums[offset][0]);
-						valid_modulus = (spec->modulus % prev_modulus) == 0;
-
-						if (valid_modulus && (offset + 1) < ndatums)
-						{
-							next_modulus = DatumGetInt32(datums[offset + 1][0]);
-							valid_modulus = (next_modulus % spec->modulus) == 0;
-						}
-					}
-
-					if (!valid_modulus)
-						ereport(ERROR,
-								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
-
-					greatest_modulus = get_greatest_modulus(boundinfo);
-					remainder = spec->remainder;
-
-					/*
-					 * Normally, the lowest remainder that could conflict with
-					 * the new partition is equal to the remainder specified
-					 * for the new partition, but when the new partition has a
-					 * modulus higher than any used so far, we need to adjust.
-					 */
-					if (remainder >= greatest_modulus)
-						remainder = remainder % greatest_modulus;
-
-					/* Check every potentially-conflicting remainder. */
-					do
-					{
-						if (boundinfo->indexes[remainder] != -1)
-						{
-							overlap = true;
-							with = boundinfo->indexes[remainder];
-							break;
-						}
-						remainder += spec->modulus;
-					} while (remainder < greatest_modulus);
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_LIST:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-
-				if (partdesc->nparts > 0)
-				{
-					ListCell   *cell;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo) ||
-							partition_bound_has_default(boundinfo)));
-
-					foreach(cell, spec->listdatums)
-					{
-						Const	   *val = castNode(Const, lfirst(cell));
-
-						if (!val->constisnull)
-						{
-							int			offset;
-							bool		equal;
-
-							offset = partition_list_bsearch(key, boundinfo,
-															val->constvalue,
-															&equal);
-							if (offset >= 0 && equal)
-							{
-								overlap = true;
-								with = boundinfo->indexes[offset];
-								break;
-							}
-						}
-						else if (partition_bound_accepts_nulls(boundinfo))
-						{
-							overlap = true;
-							with = boundinfo->null_index;
-							break;
-						}
-					}
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				PartitionRangeBound *lower,
-						   *upper;
-
-				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
-				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
-
-				/*
-				 * First check if the resulting range would be empty with
-				 * specified lower and upper bounds
-				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
-				{
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-							 errmsg("empty range bound specified for partition \"%s\"",
-									relname),
-							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
-									   get_range_partbound_string(spec->lowerdatums),
-									   get_range_partbound_string(spec->upperdatums)),
-							 parser_errposition(pstate, spec->location)));
-				}
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					int			offset;
-					bool		equal;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_has_default(boundinfo)));
-
-					/*
-					 * Test whether the new lower bound (which is treated
-					 * inclusively as part of the new partition) lies inside
-					 * an existing partition, or in a gap.
-					 *
-					 * If it's inside an existing partition, the bound at
-					 * offset + 1 will be the upper bound of that partition,
-					 * and its index will be >= 0.
-					 *
-					 * If it's in a gap, the bound at offset + 1 will be the
-					 * lower bound of the next partition, and its index will
-					 * be -1. This is also true if there is no next partition,
-					 * since the index array is initialised with an extra -1
-					 * at the end.
-					 */
-					offset = partition_range_bsearch(key, boundinfo, lower,
-													 &equal);
-
-					if (boundinfo->indexes[offset + 1] < 0)
-					{
-						/*
-						 * Check that the new partition will fit in the gap.
-						 * For it to fit, the new upper bound must be less
-						 * than or equal to the lower bound of the next
-						 * partition, if there is one.
-						 */
-						if (offset + 1 < boundinfo->ndatums)
-						{
-							int32		cmpval;
-							Datum 	   *datums;
-							PartitionRangeDatumKind *kind;
-							bool		is_lower;
-
-							datums = boundinfo->datums[offset + 1];
-							kind = boundinfo->kind[offset + 1];
-							is_lower = (boundinfo->indexes[offset + 1] == -1);
-
-							cmpval = partition_rbound_cmp(key, datums, kind,
-														  is_lower, upper);
-							if (cmpval < 0)
-							{
-								/*
-								 * The new partition overlaps with the
-								 * existing partition between offset + 1 and
-								 * offset + 2.
-								 */
-								overlap = true;
-								with = boundinfo->indexes[offset + 2];
-							}
-						}
-					}
-					else
-					{
-						/*
-						 * The new partition overlaps with the existing
-						 * partition between offset and offset + 1.
-						 */
-						overlap = true;
-						with = boundinfo->indexes[offset + 1];
-					}
-				}
-
-				break;
-			}
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	if (overlap)
-	{
-		Assert(with >= 0);
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" would overlap partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[with])),
-				 parser_errposition(pstate, spec->location)));
-	}
-}
-
-/*
  * check_default_allows_bound
  *
  * This function checks if there exists a row in the default partition that
@@ -1502,54 +349,6 @@ map_partition_varattnos(List *expr, int fromrel_varno,
 	return expr;
 }
 
-/*
- * RelationGetPartitionQual
- *
- * Returns a list of partition quals
- */
-List *
-RelationGetPartitionQual(Relation rel)
-{
-	/* Quick exit */
-	if (!rel->rd_rel->relispartition)
-		return NIL;
-
-	return generate_partition_qual(rel);
-}
-
-/*
- * get_partition_qual_relid
- *
- * Returns an expression tree describing the passed-in relation's partition
- * constraint. If there is no partition constraint returns NULL; this can
- * happen if the default partition is the only partition.
- */
-Expr *
-get_partition_qual_relid(Oid relid)
-{
-	Relation	rel = heap_open(relid, AccessShareLock);
-	Expr	   *result = NULL;
-	List	   *and_args;
-
-	/* Do the work only if this relation is a partition. */
-	if (rel->rd_rel->relispartition)
-	{
-		and_args = generate_partition_qual(rel);
-
-		if (and_args == NIL)
-			result = NULL;
-		else if (list_length(and_args) > 1)
-			result = makeBoolExpr(AND_EXPR, and_args, -1);
-		else
-			result = linitial(and_args);
-	}
-
-	/* Keep the lock. */
-	heap_close(rel, NoLock);
-
-	return result;
-}
-
 /* Module-local functions */
 
 /*
@@ -2466,644 +1265,6 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 }
 
 /*
- * generate_partition_qual
- *
- * Generate partition predicate from rel's partition bound expression. The
- * function returns a NIL list if there is no predicate.
- *
- * Result expression tree is stored CacheMemoryContext to ensure it survives
- * as long as the relcache entry. But we should be running in a less long-lived
- * working context. To avoid leaking cache memory if this routine fails partway
- * through, we build in working memory and then copy the completed structure
- * into cache memory.
- */
-static List *
-generate_partition_qual(Relation rel)
-{
-	HeapTuple	tuple;
-	MemoryContext oldcxt;
-	Datum		boundDatum;
-	bool		isnull;
-	PartitionBoundSpec *bound;
-	List	   *my_qual = NIL,
-			   *result = NIL;
-	Relation	parent;
-	bool		found_whole_row;
-
-	/* Guard against stack overflow due to overly deep partition tree */
-	check_stack_depth();
-
-	/* Quick copy */
-	if (rel->rd_partcheck != NIL)
-		return copyObject(rel->rd_partcheck);
-
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
-					   AccessShareLock);
-
-	/* Get pg_class.relpartbound */
-	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(rel));
-
-	boundDatum = SysCacheGetAttr(RELOID, tuple,
-								 Anum_pg_class_relpartbound,
-								 &isnull);
-	if (isnull)					/* should not happen */
-		elog(ERROR, "relation \"%s\" has relpartbound = null",
-			 RelationGetRelationName(rel));
-	bound = castNode(PartitionBoundSpec,
-					 stringToNode(TextDatumGetCString(boundDatum)));
-	ReleaseSysCache(tuple);
-
-	my_qual = get_qual_from_partbound(rel, parent, bound);
-
-	/* 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;
-
-	/*
-	 * Change Vars to have partition's attnos instead of the parent's. We do
-	 * this after we concatenate the parent's quals, because we want every Var
-	 * in it to bear this relation's attnos. It's safe to assume varno = 1
-	 * here.
-	 */
-	result = map_partition_varattnos(result, 1, rel, parent,
-									 &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");
-
-	/* Save a copy in the relcache */
-	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
-	rel->rd_partcheck = copyObject(result);
-	MemoryContextSwitchTo(oldcxt);
-
-	/* Keep the parent locked until commit */
-	heap_close(parent, NoLock);
-
-	return result;
-}
-
-/*
- * get_partition_for_tuple
- *		Finds partition of relation which accepts the partition key specified
- *		in values and isnull
- *
- * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
- * found or -1 if none found.
- */
-int
-get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
-{
-	int			bound_offset;
-	int			part_index = -1;
-	PartitionKey key = RelationGetPartitionKey(relation);
-	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
-
-	/* Route as appropriate based on partitioning strategy. */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				PartitionBoundInfo boundinfo = partdesc->boundinfo;
-				int			greatest_modulus = get_greatest_modulus(boundinfo);
-				uint64		rowHash = compute_hash_value(key, values, isnull);
-
-				part_index = boundinfo->indexes[rowHash % greatest_modulus];
-			}
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			if (isnull[0])
-			{
-				if (partition_bound_accepts_nulls(partdesc->boundinfo))
-					part_index = partdesc->boundinfo->null_index;
-			}
-			else
-			{
-				bool		equal = false;
-
-				bound_offset = partition_list_bsearch(key,
-													  partdesc->boundinfo,
-													  values[0], &equal);
-				if (bound_offset >= 0 && equal)
-					part_index = partdesc->boundinfo->indexes[bound_offset];
-			}
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				bool		equal = false,
-							range_partkey_has_null = false;
-				int			i;
-
-				/*
-				 * No range includes NULL, so this will be accepted by the
-				 * default partition if there is one, and otherwise rejected.
-				 */
-				for (i = 0; i < key->partnatts; i++)
-				{
-					if (isnull[i])
-					{
-						range_partkey_has_null = true;
-						break;
-					}
-				}
-
-				if (!range_partkey_has_null)
-				{
-					bound_offset = partition_range_datum_bsearch(key,
-														partdesc->boundinfo,
-														key->partnatts,
-														values,
-														&equal);
-					/*
-					 * The bound at bound_offset is less than or equal to the
-					 * tuple value, so the bound at offset+1 is the upper
-					 * bound of the partition we're looking for, if there
-					 * actually exists one.
-					 */
-					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
-				}
-			}
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/*
-	 * part_index < 0 means we failed to find a partition of this parent. Use
-	 * the default partition, if there is one.
-	 */
-	if (part_index < 0)
-		part_index = partdesc->boundinfo->default_index;
-
-	return part_index;
-}
-
-/*
- * Checks if any of the 'attnums' is a partition key attribute for rel
- *
- * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
- * partition key expression.  It's possible for a column to be both used
- * directly and as part of an expression; if that happens, *used_in_expr may
- * end up as either true or false.  That's OK for current uses of this
- * function, because *used_in_expr is only used to tailor the error message
- * text.
- */
-bool
-has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr)
-{
-	PartitionKey key;
-	int			partnatts;
-	List	   *partexprs;
-	ListCell   *partexprs_item;
-	int			i;
-
-	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return false;
-
-	key = RelationGetPartitionKey(rel);
-	partnatts = get_partition_natts(key);
-	partexprs = get_partition_exprs(key);
-
-	partexprs_item = list_head(partexprs);
-	for (i = 0; i < partnatts; i++)
-	{
-		AttrNumber	partattno = get_partition_col_attnum(key, i);
-
-		if (partattno != 0)
-		{
-			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
-							  attnums))
-			{
-				if (used_in_expr)
-					*used_in_expr = false;
-				return true;
-			}
-		}
-		else
-		{
-			/* Arbitrary expression */
-			Node	   *expr = (Node *) lfirst(partexprs_item);
-			Bitmapset  *expr_attrs = NULL;
-
-			/* Find all attributes referenced */
-			pull_varattnos(expr, 1, &expr_attrs);
-			partexprs_item = lnext(partexprs_item);
-
-			if (bms_overlap(attnums, expr_attrs))
-			{
-				if (used_in_expr)
-					*used_in_expr = true;
-				return true;
-			}
-		}
-	}
-
-	return false;
-}
-
-/*
- * qsort_partition_hbound_cmp
- *
- * We sort hash bounds by modulus, then by remainder.
- */
-static int32
-qsort_partition_hbound_cmp(const void *a, const void *b)
-{
-	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
-	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
-
-	return partition_hbound_cmp(h1->modulus, h1->remainder,
-								h2->modulus, h2->remainder);
-}
-
-/*
- * partition_hbound_cmp
- *
- * Compares modulus first, then remainder if modulus are equal.
- */
-static int32
-partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
-{
-	if (modulus1 < modulus2)
-		return -1;
-	if (modulus1 > modulus2)
-		return 1;
-	if (modulus1 == modulus2 && remainder1 != remainder2)
-		return (remainder1 > remainder2) ? 1 : -1;
-	return 0;
-}
-
-/*
- * qsort_partition_list_value_cmp
- *
- * Compare two list partition bound datums
- */
-static int32
-qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
-{
-	Datum		val1 = (*(const PartitionListValue **) a)->value,
-				val2 = (*(const PartitionListValue **) b)->value;
-	PartitionKey key = (PartitionKey) arg;
-
-	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-										   key->partcollation[0],
-										   val1, val2));
-}
-
-/*
- * make_one_range_bound
- *
- * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
- * and a flag telling whether the bound is lower or not.  Made into a function
- * because there are multiple sites that want to use this facility.
- */
-static PartitionRangeBound *
-make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
-{
-	PartitionRangeBound *bound;
-	ListCell   *lc;
-	int			i;
-
-	Assert(datums != NIL);
-
-	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
-	bound->index = index;
-	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
-	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
-													  sizeof(PartitionRangeDatumKind));
-	bound->lower = lower;
-
-	i = 0;
-	foreach(lc, datums)
-	{
-		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
-
-		/* What's contained in this range datum? */
-		bound->kind[i] = datum->kind;
-
-		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
-		{
-			Const	   *val = castNode(Const, datum->value);
-
-			if (val->constisnull)
-				elog(ERROR, "invalid range bound datum");
-			bound->datums[i] = val->constvalue;
-		}
-
-		i++;
-	}
-
-	return bound;
-}
-
-/* Used when sorting range bounds across all range partitions */
-static int32
-qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
-{
-	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
-	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
-	PartitionKey key = (PartitionKey) arg;
-
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
-}
-
-/*
- * partition_rbound_cmp
- *
- * Return for two range bounds whether the 1st one (specified in datums1,
- * kind1, and lower1) is <, =, or > the bound specified in *b2.
- *
- * Note that if the values of the two range bounds compare equal, then we take
- * into account whether they are upper or lower bounds, and an upper bound is
- * considered to be smaller than a lower bound. This is important to the way
- * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
- * structure, which only stores the upper bound of a common boundary between
- * two contiguous partitions.
- */
-static int32
-partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2)
-{
-	int32		cmpval = 0;		/* placate compiler */
-	int			i;
-	Datum	   *datums2 = b2->datums;
-	PartitionRangeDatumKind *kind2 = b2->kind;
-	bool		lower2 = b2->lower;
-
-	for (i = 0; i < key->partnatts; i++)
-	{
-		/*
-		 * First, handle cases where the column is unbounded, which should not
-		 * invoke the comparison procedure, and should not consider any later
-		 * columns. Note that the PartitionRangeDatumKind enum elements
-		 * compare the same way as the values they represent.
-		 */
-		if (kind1[i] < kind2[i])
-			return -1;
-		else if (kind1[i] > kind2[i])
-			return 1;
-		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
-
-			/*
-			 * The column bounds are both MINVALUE or both MAXVALUE. No later
-			 * columns should be considered, but we still need to compare
-			 * whether they are upper or lower bounds.
-			 */
-			break;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 datums1[i],
-												 datums2[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	/*
-	 * If the comparison is anything other than equal, we're done. If they
-	 * compare equal though, we still have to consider whether the boundaries
-	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
-	 * two.
-	 */
-	if (cmpval == 0 && lower1 != lower2)
-		cmpval = lower1 ? 1 : -1;
-
-	return cmpval;
-}
-
-/*
- * partition_rbound_datum_cmp
- *
- * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
- * is <, =, or > partition key of tuple (tuple_datums)
- */
-static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums)
-{
-	int			i;
-	int32		cmpval = -1;
-
-	for (i = 0; i < n_tuple_datums; i++)
-	{
-		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
-			return -1;
-		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
-			return 1;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
-												 rb_datums[i],
-												 tuple_datums[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	return cmpval;
-}
-
-/*
- * partition_list_bsearch
- *		Returns the index of the greatest bound datum that is less than equal
- * 		to the given value or -1 if all of the bound datums are greater
- *
- * *is_equal is set to true if the bound datum at the returned index is equal
- * to the input value.
- */
-static int
-partition_list_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-												 key->partcollation[0],
-												 boundinfo->datums[mid][0],
-												 value));
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given range bound or -1 if all of the range bounds are
- *		greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input range bound
- */
-static int
-partition_range_bsearch(PartitionKey key,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_cmp(key,
-									  boundinfo->datums[mid],
-									  boundinfo->kind[mid],
-									  (boundinfo->indexes[mid] == -1),
-									  probe);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given tuple or -1 if all of the range bounds are greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input tuple.
- */
-static int
-partition_range_datum_bsearch(PartitionKey key,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_datum_cmp(key,
-											boundinfo->datums[mid],
-											boundinfo->kind[mid],
-											values,
-											nvalues);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_hash_bsearch
- *		Returns the index of the greatest (modulus, remainder) pair that is
- *		less than or equal to the given (modulus, remainder) pair or -1 if
- *		all of them are greater
- */
-static int
-partition_hash_bsearch(PartitionKey key,
-					   PartitionBoundInfo boundinfo,
-					   int modulus, int remainder)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval,
-					bound_modulus,
-					bound_remainder;
-
-		mid = (lo + hi + 1) / 2;
-		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
-		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
-		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
-									  modulus, remainder);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-
-			if (cmpval == 0)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * get_default_oid_from_partdesc
- *
- * Given a partition descriptor, return the OID of the default partition, if
- * one exists; else, return InvalidOid.
- */
-Oid
-get_default_oid_from_partdesc(PartitionDesc partdesc)
-{
-	if (partdesc && partdesc->boundinfo &&
-		partition_bound_has_default(partdesc->boundinfo))
-		return partdesc->oids[partdesc->boundinfo->default_index];
-
-	return InvalidOid;
-}
-
-/*
  * get_default_partition_oid
  *
  * Given a relation OID, return the OID of the default partition, if one
@@ -3189,99 +1350,6 @@ get_proposed_default_constraint(List *new_part_constraints)
 }
 
 /*
- * get_partition_bound_num_indexes
- *
- * Returns the number of the entries in the partition bound indexes array.
- */
-static int
-get_partition_bound_num_indexes(PartitionBoundInfo bound)
-{
-	int			num_indexes;
-
-	Assert(bound);
-
-	switch (bound->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-
-			/*
-			 * The number of the entries in the indexes array is same as the
-			 * greatest modulus.
-			 */
-			num_indexes = get_greatest_modulus(bound);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			num_indexes = bound->ndatums;
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			/* Range partitioned table has an extra index. */
-			num_indexes = bound->ndatums + 1;
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) bound->strategy);
-	}
-
-	return num_indexes;
-}
-
-/*
- * get_greatest_modulus
- *
- * Returns the greatest modulus of the hash partition bound. The greatest
- * modulus will be at the end of the datums array because hash partitions are
- * arranged in the ascending order of their modulus and remainders.
- */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
-{
-	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
-	Assert(bound->datums && bound->ndatums > 0);
-	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
-
-	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
-}
-
-/*
- * compute_hash_value
- *
- * Compute the hash value for given not null partition key values.
- */
-static uint64
-compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
-{
-	int			i;
-	int			nkeys = key->partnatts;
-	uint64		rowHash = 0;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-
-	for (i = 0; i < nkeys; i++)
-	{
-		if (!isnull[i])
-		{
-			Datum		hash;
-
-			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
-
-			/*
-			 * Compute hash for each datum value by calling respective
-			 * datatype-specific hash functions of each partition key
-			 * attribute.
-			 */
-			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	return rowHash;
-}
-
-/*
  * satisfies_hash_partition
  *
  * This is an SQL-callable function for use in hash partition constraints.
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 5d3e923cca..b17abb5c7d 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -42,7 +42,6 @@
 #include "access/transam.h"
 #include "access/xact.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_publication.h"
 #include "commands/matview.h"
 #include "commands/trigger.h"
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 4048c3ebc6..cc77ba3701 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -20,6 +20,7 @@
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index f74afdb4dd..7164a09995 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -15,13 +15,13 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/prep.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/memutils.h"
 
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b586f941a8..ad2d750bd4 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,7 +33,6 @@
 #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"
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 60f21711f4..605493d04a 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,7 +27,6 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_statistic_ext.h"
 #include "foreign/fdwapi.h"
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 5c368321e6..74f8e0a04d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -17,7 +17,6 @@
 #include <limits.h>
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
@@ -28,6 +27,7 @@
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
+#include "utils/partcache.h"
 
 
 typedef struct JoinHashEntry
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index ba9fab4582..0797e258b3 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -24,7 +24,6 @@
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
-#include "catalog/partition.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_authid.h"
diff --git a/src/backend/utils/cache/Makefile b/src/backend/utils/cache/Makefile
index a943f8ea4b..94511eaf54 100644
--- a/src/backend/utils/cache/Makefile
+++ b/src/backend/utils/cache/Makefile
@@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
-	relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
-	typcache.o ts_cache.o
+OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o partcache.o \
+    relcache.o relmapper.o relfilenodemap.o spccache.o syscache.o \
+    lsyscache.o typcache.o ts_cache.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
new file mode 100644
index 0000000000..8d40f3b2ba
--- /dev/null
+++ b/src/backend/utils/cache/partcache.c
@@ -0,0 +1,2116 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.c
+ *		Partitioning related cache data structures and manipulation functions
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *		  src/backend/utils/cache/partcache.c
+ *
+ *-------------------------------------------------------------------------
+*/
+
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "access/nbtree.h"
+#include "access/sysattr.h"
+#include "catalog/partition.h"
+#include "catalog/pg_inherits_fn.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
+#include "utils/builtins.h"
+#include "utils/datum.h"
+#include "utils/hashutils.h"
+#include "utils/lsyscache.h"
+#include "utils/memutils.h"
+#include "utils/ruleutils.h"
+#include "utils/syscache.h"
+
+/*
+ * Partition bounds are put into using one of the following structs when they
+ * are first read into the backend memory from the catalog wherein they are
+ * stored in in their parser represenation.  Especially, the bound comparison
+ * and sort functions expect the bounds passed to them to be of this form.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int			modulus;
+	int			remainder;
+	int			index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+	int			index;
+	Datum		value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+	int			index;
+	Datum	   *datums;			/* range bound datums */
+	PartitionRangeDatumKind *kind;	/* the kind of each datum */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+static List *generate_partition_qual(Relation rel);
+
+static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
+					 int remainder2);
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
+static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
+							   void *arg);
+static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
+					 List *datums, bool lower);
+static int32 partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2);
+static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
+						   void *arg);
+
+static int partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal);
+static int partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal);
+static int32 partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums);
+static int partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal);
+static int partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder);
+
+static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
+
+/*
+ * RelationBuildPartitionKey
+ *		Build and attach to relcache partition key data of relation
+ *
+ * Partitioning key data is a complex structure; to avoid complicated logic to
+ * free individual elements whenever the relcache entry is flushed, we give it
+ * its own memory context, child of CacheMemoryContext, which can easily be
+ * deleted on its own.  To avoid leaking memory in that context in case of an
+ * error partway through this function, the context is initially created as a
+ * child of CurTransactionContext and only re-parented to CacheMemoryContext
+ * at the end, when no further errors are possible.  Also, we don't make this
+ * context the current context except in very brief code sections, out of fear
+ * that some of our callees allocate memory on their own which would be leaked
+ * permanently.
+ */
+void
+RelationBuildPartitionKey(Relation relation)
+{
+	Form_pg_partitioned_table form;
+	HeapTuple	tuple;
+	bool		isnull;
+	int			i;
+	PartitionKey key;
+	AttrNumber *attrs;
+	oidvector  *opclass;
+	oidvector  *collation;
+	ListCell   *partexprs_item;
+	Datum		datum;
+	MemoryContext partkeycxt,
+				oldcxt;
+	int16		procnum;
+
+	tuple = SearchSysCache1(PARTRELID,
+							ObjectIdGetDatum(RelationGetRelid(relation)));
+
+	/*
+	 * The following happens when we have created our pg_class entry but not
+	 * the pg_partitioned_table entry yet.
+	 */
+	if (!HeapTupleIsValid(tuple))
+		return;
+
+	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
+											   RelationGetRelationName(relation),
+											   MEMCONTEXT_COPY_NAME,
+											   ALLOCSET_SMALL_SIZES);
+
+	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
+												sizeof(PartitionKeyData));
+
+	/* Fixed-length attributes */
+	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	key->strategy = form->partstrat;
+	key->partnatts = form->partnatts;
+
+	/*
+	 * We can rely on the first variable-length attribute being mapped to the
+	 * relevant field of the catalog's C struct, because all previous
+	 * attributes are non-nullable and fixed-length.
+	 */
+	attrs = form->partattrs.values;
+
+	/* But use the hard way to retrieve further variable-length attributes */
+	/* Operator class */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partclass, &isnull);
+	Assert(!isnull);
+	opclass = (oidvector *) DatumGetPointer(datum);
+
+	/* Collation */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partcollation, &isnull);
+	Assert(!isnull);
+	collation = (oidvector *) DatumGetPointer(datum);
+
+	/* Expressions */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partexprs, &isnull);
+	if (!isnull)
+	{
+		char	   *exprString;
+		Node	   *expr;
+
+		exprString = TextDatumGetCString(datum);
+		expr = stringToNode(exprString);
+		pfree(exprString);
+
+		/*
+		 * Run the expressions through const-simplification since the planner
+		 * will be comparing them to similarly-processed qual clause operands,
+		 * and may fail to detect valid matches without this step; fix
+		 * opfuncids while at it.  We don't need to bother with
+		 * canonicalize_qual() though, because partition expressions are not
+		 * full-fledged qualification clauses.
+		 */
+		expr = eval_const_expressions(NULL, expr);
+		fix_opfuncids(expr);
+
+		oldcxt = MemoryContextSwitchTo(partkeycxt);
+		key->partexprs = (List *) copyObject(expr);
+		MemoryContextSwitchTo(oldcxt);
+	}
+
+	oldcxt = MemoryContextSwitchTo(partkeycxt);
+	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
+	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
+
+	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+
+	/* Gather type and collation info as well */
+	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
+	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
+	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
+	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
+	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	MemoryContextSwitchTo(oldcxt);
+
+	/* determine support function number to search for */
+	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
+		HASHEXTENDED_PROC : BTORDER_PROC;
+
+	/* Copy partattrs and fill other per-attribute info */
+	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		AttrNumber	attno = key->partattrs[i];
+		HeapTuple	opclasstup;
+		Form_pg_opclass opclassform;
+		Oid			funcid;
+
+		/* Collect opfamily information */
+		opclasstup = SearchSysCache1(CLAOID,
+									 ObjectIdGetDatum(opclass->values[i]));
+		if (!HeapTupleIsValid(opclasstup))
+			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
+
+		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
+		key->partopfamily[i] = opclassform->opcfamily;
+		key->partopcintype[i] = opclassform->opcintype;
+
+		/* Get a support function for the specified opfamily and datatypes */
+		funcid = get_opfamily_proc(opclassform->opcfamily,
+								   opclassform->opcintype,
+								   opclassform->opcintype,
+								   procnum);
+		if (!OidIsValid(funcid))
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
+							NameStr(opclassform->opcname),
+							(key->strategy == PARTITION_STRATEGY_HASH) ?
+							"hash" : "btree",
+							procnum,
+							format_type_be(opclassform->opcintype))));
+
+		fmgr_info(funcid, &key->partsupfunc[i]);
+
+		/* Collation */
+		key->partcollation[i] = collation->values[i];
+
+		/* Collect type information */
+		if (attno != 0)
+		{
+			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
+
+			key->parttypid[i] = att->atttypid;
+			key->parttypmod[i] = att->atttypmod;
+			key->parttypcoll[i] = att->attcollation;
+		}
+		else
+		{
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			key->parttypid[i] = exprType(lfirst(partexprs_item));
+			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
+			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
+
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(key->parttypid[i],
+							 &key->parttyplen[i],
+							 &key->parttypbyval[i],
+							 &key->parttypalign[i]);
+
+		ReleaseSysCache(opclasstup);
+	}
+
+	ReleaseSysCache(tuple);
+
+	/*
+	 * Success --- reparent our context and make the relcache point to the
+	 * newly constructed key
+	 */
+	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
+	relation->rd_partkeycxt = partkeycxt;
+	relation->rd_partkey = key;
+}
+
+/*
+ * RelationBuildPartitionDesc
+ *		Form rel's partition descriptor
+ *
+ * Not flushed from the cache by RelationClearRelation() unless changed because
+ * of addition or removal of partition.
+ */
+void
+RelationBuildPartitionDesc(Relation rel)
+{
+	List	   *inhoids,
+			   *partoids;
+	Oid		   *oids = NULL;
+	List	   *boundspecs = NIL;
+	ListCell   *cell;
+	int			i,
+				nparts;
+	PartitionKey key = RelationGetPartitionKey(rel);
+	PartitionDesc result;
+	MemoryContext oldcxt;
+
+	int			ndatums = 0;
+	int			default_index = -1;
+
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
+	/* List partitioning specific */
+	PartitionListValue **all_values = NULL;
+	int			null_index = -1;
+
+	/* Range partitioning specific */
+	PartitionRangeBound **rbounds = NULL;
+
+	/*
+	 * The following could happen in situations where rel has a pg_class entry
+	 * but not the pg_partitioned_table entry yet.
+	 */
+	if (key == NULL)
+		return;
+
+	/* Get partition oids from pg_inherits */
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+
+	/* Collect bound spec nodes in a list */
+	i = 0;
+	partoids = NIL;
+	foreach(cell, inhoids)
+	{
+		Oid			inhrelid = lfirst_oid(cell);
+		HeapTuple	tuple;
+		Datum		datum;
+		bool		isnull;
+		Node	   *boundspec;
+
+		tuple = SearchSysCache1(RELOID, inhrelid);
+		if (!HeapTupleIsValid(tuple))
+			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+		/*
+		 * It is possible that the pg_class tuple of a partition has not been
+		 * updated yet to set its relpartbound field.  The only case where
+		 * this happens is when we open the parent relation to check using its
+		 * partition descriptor that a new partition's bound does not overlap
+		 * some existing partition.
+		 */
+		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		{
+			ReleaseSysCache(tuple);
+			continue;
+		}
+
+		datum = SysCacheGetAttr(RELOID, tuple,
+								Anum_pg_class_relpartbound,
+								&isnull);
+		Assert(!isnull);
+		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+		/*
+		 * Sanity check: If the PartitionBoundSpec says this is the default
+		 * partition, its OID should correspond to whatever's stored in
+		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 */
+		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		{
+			Oid			partdefid;
+
+			partdefid = get_default_partition_oid(RelationGetRelid(rel));
+			if (partdefid != inhrelid)
+				elog(ERROR, "expected partdefid %u, but got %u",
+					 inhrelid, partdefid);
+		}
+
+		boundspecs = lappend(boundspecs, boundspec);
+		partoids = lappend_oid(partoids, inhrelid);
+		ReleaseSysCache(tuple);
+	}
+
+	nparts = list_length(partoids);
+
+	if (nparts > 0)
+	{
+		oids = (Oid *) palloc(nparts * sizeof(Oid));
+		i = 0;
+		foreach(cell, partoids)
+			oids[i++] = lfirst_oid(cell);
+
+		/* Convert from node to the internal representation */
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **)
+				palloc(nparts * sizeof(PartitionHashBound *));
+
+			i = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
+		{
+			List	   *non_null_values = NIL;
+
+			/*
+			 * Create a unified list of non-null values across all partitions.
+			 */
+			i = 0;
+			null_index = -1;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				ListCell   *c;
+
+				if (spec->strategy != PARTITION_STRATEGY_LIST)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
+				foreach(c, spec->listdatums)
+				{
+					Const	   *val = castNode(Const, lfirst(c));
+					PartitionListValue *list_value = NULL;
+
+					if (!val->constisnull)
+					{
+						list_value = (PartitionListValue *)
+							palloc0(sizeof(PartitionListValue));
+						list_value->index = i;
+						list_value->value = val->constvalue;
+					}
+					else
+					{
+						/*
+						 * Never put a null into the values array, flag
+						 * instead for the code further down below where we
+						 * construct the actual relcache struct.
+						 */
+						if (null_index != -1)
+							elog(ERROR, "found null more than once");
+						null_index = i;
+					}
+
+					if (list_value)
+						non_null_values = lappend(non_null_values,
+												  list_value);
+				}
+
+				i++;
+			}
+
+			ndatums = list_length(non_null_values);
+
+			/*
+			 * Collect all list values in one array. Alongside the value, we
+			 * also save the index of partition the value comes from.
+			 */
+			all_values = (PartitionListValue **) palloc(ndatums *
+														sizeof(PartitionListValue *));
+			i = 0;
+			foreach(cell, non_null_values)
+			{
+				PartitionListValue *src = lfirst(cell);
+
+				all_values[i] = (PartitionListValue *)
+					palloc(sizeof(PartitionListValue));
+				all_values[i]->value = src->value;
+				all_values[i]->index = src->index;
+				i++;
+			}
+
+			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
+					  qsort_partition_list_value_cmp, (void *) key);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_RANGE)
+		{
+			int			k;
+			PartitionRangeBound **all_bounds,
+					   *prev;
+
+			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
+														  sizeof(PartitionRangeBound *));
+
+			/*
+			 * Create a unified list of range bounds across all the
+			 * partitions.
+			 */
+			i = ndatums = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				PartitionRangeBound *lower,
+						   *upper;
+
+				if (spec->strategy != PARTITION_STRATEGY_RANGE)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the allbounds array
+				 * for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i++;
+					continue;
+				}
+
+				lower = make_one_range_bound(key, i, spec->lowerdatums,
+											 true);
+				upper = make_one_range_bound(key, i, spec->upperdatums,
+											 false);
+				all_bounds[ndatums++] = lower;
+				all_bounds[ndatums++] = upper;
+				i++;
+			}
+
+			Assert(ndatums == nparts * 2 ||
+				   (default_index != -1 && ndatums == (nparts - 1) * 2));
+
+			/* Sort all the bounds in ascending order */
+			qsort_arg(all_bounds, ndatums,
+					  sizeof(PartitionRangeBound *),
+					  qsort_partition_rbound_cmp,
+					  (void *) key);
+
+			/* Save distinct bounds from all_bounds into rbounds. */
+			rbounds = (PartitionRangeBound **)
+				palloc(ndatums * sizeof(PartitionRangeBound *));
+			k = 0;
+			prev = NULL;
+			for (i = 0; i < ndatums; i++)
+			{
+				PartitionRangeBound *cur = all_bounds[i];
+				bool		is_distinct = false;
+				int			j;
+
+				/* Is the current bound distinct from the previous one? */
+				for (j = 0; j < key->partnatts; j++)
+				{
+					Datum		cmpval;
+
+					if (prev == NULL || cur->kind[j] != prev->kind[j])
+					{
+						is_distinct = true;
+						break;
+					}
+
+					/*
+					 * If the bounds are both MINVALUE or MAXVALUE, stop now
+					 * and treat them as equal, since any values after this
+					 * point must be ignored.
+					 */
+					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
+						break;
+
+					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
+											   key->partcollation[j],
+											   cur->datums[j],
+											   prev->datums[j]);
+					if (DatumGetInt32(cmpval) != 0)
+					{
+						is_distinct = true;
+						break;
+					}
+				}
+
+				/*
+				 * Only if the bound is distinct save it into a temporary
+				 * array i.e. rbounds which is later copied into boundinfo
+				 * datums array.
+				 */
+				if (is_distinct)
+					rbounds[k++] = all_bounds[i];
+
+				prev = cur;
+			}
+
+			/* Update ndatums to hold the count of distinct datums. */
+			ndatums = k;
+		}
+		else
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/* Now build the actual relcache partition descriptor */
+	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
+												  RelationGetRelationName(rel),
+												  MEMCONTEXT_COPY_NAME,
+												  ALLOCSET_DEFAULT_SIZES);
+	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+
+	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
+	result->nparts = nparts;
+	if (nparts > 0)
+	{
+		PartitionBoundInfo boundinfo;
+		int		   *mapping;
+		int			next_index = 0;
+
+		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
+
+		boundinfo = (PartitionBoundInfoData *)
+			palloc0(sizeof(PartitionBoundInfoData));
+		boundinfo->strategy = key->strategy;
+		boundinfo->default_index = -1;
+		boundinfo->ndatums = ndatums;
+		boundinfo->null_index = -1;
+		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
+
+		/* Initialize mapping array with invalid values */
+		mapping = (int *) palloc(sizeof(int) * nparts);
+		for (i = 0; i < nparts; i++)
+			mapping[i] = -1;
+
+		switch (key->strategy)
+		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+
+					for (i = 0; i < greatest_modulus; i++)
+						boundinfo->indexes[i] = -1;
+
+					for (i = 0; i < nparts; i++)
+					{
+						int			modulus = hbounds[i]->modulus;
+						int			remainder = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(modulus);
+						boundinfo->datums[i][1] = Int32GetDatum(remainder);
+
+						while (remainder < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[remainder] == -1);
+							boundinfo->indexes[remainder] = i;
+							remainder += modulus;
+						}
+
+						mapping[hbounds[i]->index] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
+			case PARTITION_STRATEGY_LIST:
+				{
+					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
+
+					/*
+					 * Copy values.  Indexes of individual values are mapped
+					 * to canonical values so that they match for any two list
+					 * partitioned tables with same number of partitions and
+					 * same lists per partition.  One way to canonicalize is
+					 * to assign the index in all_values[] of the smallest
+					 * value of each partition, as the index of all of the
+					 * partition's values.
+					 */
+					for (i = 0; i < ndatums; i++)
+					{
+						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
+						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
+															key->parttypbyval[0],
+															key->parttyplen[0]);
+
+						/* If the old index has no mapping, assign one */
+						if (mapping[all_values[i]->index] == -1)
+							mapping[all_values[i]->index] = next_index++;
+
+						boundinfo->indexes[i] = mapping[all_values[i]->index];
+					}
+
+					/*
+					 * If null-accepting partition has no mapped index yet,
+					 * assign one.  This could happen if such partition
+					 * accepts only null and hence not covered in the above
+					 * loop which only handled non-null values.
+					 */
+					if (null_index != -1)
+					{
+						Assert(null_index >= 0);
+						if (mapping[null_index] == -1)
+							mapping[null_index] = next_index++;
+						boundinfo->null_index = mapping[null_index];
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any value not
+						 * specified in the lists of other partitions, hence
+						 * it should not get mapped index while assigning
+						 * those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+
+					/* All partition must now have a valid mapping */
+					Assert(next_index == nparts);
+					break;
+				}
+
+			case PARTITION_STRATEGY_RANGE:
+				{
+					boundinfo->kind = (PartitionRangeDatumKind **)
+						palloc(ndatums *
+							   sizeof(PartitionRangeDatumKind *));
+					boundinfo->indexes = (int *) palloc((ndatums + 1) *
+														sizeof(int));
+
+					for (i = 0; i < ndatums; i++)
+					{
+						int			j;
+
+						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
+																sizeof(Datum));
+						boundinfo->kind[i] = (PartitionRangeDatumKind *)
+							palloc(key->partnatts *
+								   sizeof(PartitionRangeDatumKind));
+						for (j = 0; j < key->partnatts; j++)
+						{
+							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
+								boundinfo->datums[i][j] =
+									datumCopy(rbounds[i]->datums[j],
+											  key->parttypbyval[j],
+											  key->parttyplen[j]);
+							boundinfo->kind[i][j] = rbounds[i]->kind[j];
+						}
+
+						/*
+						 * There is no mapping for invalid indexes.
+						 *
+						 * Any lower bounds in the rbounds array have invalid
+						 * indexes assigned, because the values between the
+						 * previous bound (if there is one) and this (lower)
+						 * bound are not part of the range of any existing
+						 * partition.
+						 */
+						if (rbounds[i]->lower)
+							boundinfo->indexes[i] = -1;
+						else
+						{
+							int			orig_index = rbounds[i]->index;
+
+							/* If the old index has no mapping, assign one */
+							if (mapping[orig_index] == -1)
+								mapping[orig_index] = next_index++;
+
+							boundinfo->indexes[i] = mapping[orig_index];
+						}
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						Assert(default_index >= 0 && mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					boundinfo->indexes[i] = -1;
+					break;
+				}
+
+			default:
+				elog(ERROR, "unexpected partition strategy: %d",
+					 (int) key->strategy);
+		}
+
+		result->boundinfo = boundinfo;
+
+		/*
+		 * Now assign OIDs from the original array into mapped indexes of the
+		 * result array.  Order of OIDs in the former is defined by the
+		 * catalog scan that retrieved them, whereas that in the latter is
+		 * defined by canonicalized representation of the partition bounds.
+		 */
+		for (i = 0; i < nparts; i++)
+			result->oids[mapping[i]] = oids[i];
+		pfree(mapping);
+	}
+
+	MemoryContextSwitchTo(oldcxt);
+	rel->rd_partdesc = result;
+}
+
+/*
+ * Are two partition bound collections logically equal?
+ *
+ * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
+ * This is also useful when b1 and b2 are bound collections of two separate
+ * relations, respectively, because PartitionBoundInfo is a canonical
+ * representation of partition bounds.
+ */
+bool
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
+					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			i;
+
+	if (b1->strategy != b2->strategy)
+		return false;
+
+	if (b1->ndatums != b2->ndatums)
+		return false;
+
+	if (b1->null_index != b2->null_index)
+		return false;
+
+	if (b1->default_index != b2->default_index)
+		return false;
+
+	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	{
+		int			greatest_modulus = get_greatest_modulus(b1);
+
+		/*
+		 * If two hash partitioned tables have different greatest moduli,
+		 * their partition schemes don't match.
+		 */
+		if (greatest_modulus != get_greatest_modulus(b2))
+			return false;
+
+		/*
+		 * We arrange the partitions in the ascending order of their modulus
+		 * and remainders.  Also every modulus is factor of next larger
+		 * modulus.  Therefore we can safely store index of a given partition
+		 * in indexes array at remainder of that partition.  Also entries at
+		 * (remainder + N * modulus) positions in indexes array are all same
+		 * for (modulus, remainder) specification for any partition.  Thus
+		 * datums array from both the given bounds are same, if and only if
+		 * their indexes array will be same.  So, it suffices to compare
+		 * indexes array.
+		 */
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Nonetheless make sure that the bounds are indeed same when the
+		 * indexes match.  Hash partition bound stores modulus and remainder
+		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			Assert((b1->datums[i][0] == b2->datums[i][0] &&
+					b1->datums[i][1] == b2->datums[i][1]));
+#endif
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
+		{
+			int			j;
+
+			for (j = 0; j < partnatts; j++)
+			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->kind != NULL)
+				{
+					/* The different kinds of bound all differ from each other */
+					if (b1->kind[i][j] != b2->kind[i][j])
+						return false;
+
+					/*
+					 * Non-finite bounds are equal without further
+					 * examination.
+					 */
+					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
+						continue;
+				}
+
+				/*
+				 * Compare the actual values. Note that it would be both
+				 * incorrect and unsafe to invoke the comparison operator
+				 * derived from the partitioning specification here.  It would
+				 * be incorrect because we want the relcache entry to be
+				 * updated for ANY change to the partition bounds, not just
+				 * those that the partitioning operator thinks are
+				 * significant.  It would be unsafe because we might reach
+				 * this code in the context of an aborted transaction, and an
+				 * arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be
+				 * safe.
+				 */
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  parttypbyval[j], parttyplen[j]))
+					return false;
+			}
+
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+		}
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
+			return false;
+	}
+	return true;
+}
+
+/*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specification.
+ */
+PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo dest;
+	int			i;
+	int			ndatums;
+	int			partnatts;
+	int			num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	num_indexes = get_partition_bound_num_indexes(src);
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+														 sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+															   sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int			j;
+
+		/*
+		 * For a corresponding to hash partition, datums array will have two
+		 * elements - modulus and remainder.
+		 */
+		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
+		int			natts = hash_part ? 2 : partnatts;
+
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+
+		for (j = 0; j < natts; j++)
+		{
+			bool		byval;
+			int			typlen;
+
+			if (hash_part)
+			{
+				typlen = sizeof(int32); /* Always int4 */
+				byval = true;	/* int4 is pass-by-value */
+			}
+			else
+			{
+				byval = key->parttypbyval[j];
+				typlen = key->parttyplen[j];
+			}
+
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   byval, typlen);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
+ * check_new_partition_bound
+ *
+ * Checks if the new partition's bound overlaps any of the existing partitions
+ * of parent.  Also performs additional checks as necessary per strategy.
+ */
+void
+check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
+	ParseState *pstate = make_parsestate(NULL);
+	int			with = -1;
+	bool		overlap = false;
+
+	if (spec->is_default)
+	{
+		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+			return;
+
+		/* Default partition already exists, error out. */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+				 parser_errposition(pstate, spec->location)));
+	}
+
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					Datum	  **datums = boundinfo->datums;
+					int			ndatums = boundinfo->ndatums;
+					int			greatest_modulus;
+					int			remainder;
+					int			offset;
+					bool		valid_modulus = true;
+					int			prev_modulus,	/* Previous largest modulus */
+								next_modulus;	/* Next largest modulus */
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.
+					 *
+					 * Get the greatest (modulus, remainder) pair contained in
+					 * boundinfo->datums that is less than or equal to the
+					 * (spec->modulus, spec->remainder) pair.
+					 */
+					offset = partition_hash_bsearch(key, boundinfo,
+													spec->modulus,
+													spec->remainder);
+					if (offset < 0)
+					{
+						next_modulus = DatumGetInt32(datums[0][0]);
+						valid_modulus = (next_modulus % spec->modulus) == 0;
+					}
+					else
+					{
+						prev_modulus = DatumGetInt32(datums[offset][0]);
+						valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+						if (valid_modulus && (offset + 1) < ndatums)
+						{
+							next_modulus = DatumGetInt32(datums[offset + 1][0]);
+							valid_modulus = (next_modulus % spec->modulus) == 0;
+						}
+					}
+
+					if (!valid_modulus)
+						ereport(ERROR,
+								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
+
+					greatest_modulus = get_greatest_modulus(boundinfo);
+					remainder = spec->remainder;
+
+					/*
+					 * Normally, the lowest remainder that could conflict with
+					 * the new partition is equal to the remainder specified
+					 * for the new partition, but when the new partition has a
+					 * modulus higher than any used so far, we need to adjust.
+					 */
+					if (remainder >= greatest_modulus)
+						remainder = remainder % greatest_modulus;
+
+					/* Check every potentially-conflicting remainder. */
+					do
+					{
+						if (boundinfo->indexes[remainder] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[remainder];
+							break;
+						}
+						remainder += spec->modulus;
+					} while (remainder < greatest_modulus);
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_LIST:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+
+				if (partdesc->nparts > 0)
+				{
+					ListCell   *cell;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					foreach(cell, spec->listdatums)
+					{
+						Const	   *val = castNode(Const, lfirst(cell));
+
+						if (!val->constisnull)
+						{
+							int			offset;
+							bool		equal;
+
+							offset = partition_list_bsearch(key, boundinfo,
+															val->constvalue,
+															&equal);
+							if (offset >= 0 && equal)
+							{
+								overlap = true;
+								with = boundinfo->indexes[offset];
+								break;
+							}
+						}
+						else if (partition_bound_accepts_nulls(boundinfo))
+						{
+							overlap = true;
+							with = boundinfo->null_index;
+							break;
+						}
+					}
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				PartitionRangeBound *lower,
+						   *upper;
+
+				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
+				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
+
+				/*
+				 * First check if the resulting range would be empty with
+				 * specified lower and upper bounds
+				 */
+				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
+										 upper) >= 0)
+				{
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("empty range bound specified for partition \"%s\"",
+									relname),
+							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
+									   get_range_partbound_string(spec->lowerdatums),
+									   get_range_partbound_string(spec->upperdatums)),
+							 parser_errposition(pstate, spec->location)));
+				}
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					int			offset;
+					bool		equal;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Test whether the new lower bound (which is treated
+					 * inclusively as part of the new partition) lies inside
+					 * an existing partition, or in a gap.
+					 *
+					 * If it's inside an existing partition, the bound at
+					 * offset + 1 will be the upper bound of that partition,
+					 * and its index will be >= 0.
+					 *
+					 * If it's in a gap, the bound at offset + 1 will be the
+					 * lower bound of the next partition, and its index will
+					 * be -1. This is also true if there is no next partition,
+					 * since the index array is initialised with an extra -1
+					 * at the end.
+					 */
+					offset = partition_range_bsearch(key, boundinfo, lower,
+													 &equal);
+
+					if (boundinfo->indexes[offset + 1] < 0)
+					{
+						/*
+						 * Check that the new partition will fit in the gap.
+						 * For it to fit, the new upper bound must be less
+						 * than or equal to the lower bound of the next
+						 * partition, if there is one.
+						 */
+						if (offset + 1 < boundinfo->ndatums)
+						{
+							int32		cmpval;
+							Datum 	   *datums;
+							PartitionRangeDatumKind *kind;
+							bool		is_lower;
+
+							datums = boundinfo->datums[offset + 1];
+							kind = boundinfo->kind[offset + 1];
+							is_lower = (boundinfo->indexes[offset + 1] == -1);
+
+							cmpval = partition_rbound_cmp(key, datums, kind,
+														  is_lower, upper);
+							if (cmpval < 0)
+							{
+								/*
+								 * The new partition overlaps with the
+								 * existing partition between offset + 1 and
+								 * offset + 2.
+								 */
+								overlap = true;
+								with = boundinfo->indexes[offset + 2];
+							}
+						}
+					}
+					else
+					{
+						/*
+						 * The new partition overlaps with the existing
+						 * partition between offset and offset + 1.
+						 */
+						overlap = true;
+						with = boundinfo->indexes[offset + 1];
+					}
+				}
+
+				break;
+			}
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	if (overlap)
+	{
+		Assert(with >= 0);
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" would overlap partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[with])),
+				 parser_errposition(pstate, spec->location)));
+	}
+}
+
+/*
+ * RelationGetPartitionQual
+ *
+ * Returns a list of partition quals
+ */
+List *
+RelationGetPartitionQual(Relation rel)
+{
+	/* Quick exit */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
+	return generate_partition_qual(rel);
+}
+
+/*
+ * get_partition_qual_relid
+ *
+ * Returns an expression tree describing the passed-in relation's partition
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
+ */
+Expr *
+get_partition_qual_relid(Oid relid)
+{
+	Relation	rel = heap_open(relid, AccessShareLock);
+	Expr	   *result = NULL;
+	List	   *and_args;
+
+	/* Do the work only if this relation is a partition. */
+	if (rel->rd_rel->relispartition)
+	{
+		and_args = generate_partition_qual(rel);
+
+		if (and_args == NIL)
+			result = NULL;
+		else if (list_length(and_args) > 1)
+			result = makeBoolExpr(AND_EXPR, and_args, -1);
+		else
+			result = linitial(and_args);
+	}
+
+	/* Keep the lock. */
+	heap_close(rel, NoLock);
+
+	return result;
+}
+
+/*
+ * Checks if any of the 'attnums' is a partition key attribute for rel
+ *
+ * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
+ * partition key expression.  It's possible for a column to be both used
+ * directly and as part of an expression; if that happens, *used_in_expr may
+ * end up as either true or false.  That's OK for current uses of this
+ * function, because *used_in_expr is only used to tailor the error message
+ * text.
+ */
+bool
+has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr)
+{
+	PartitionKey key;
+	int			partnatts;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	int			i;
+
+	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		return false;
+
+	key = RelationGetPartitionKey(rel);
+	partnatts = get_partition_natts(key);
+	partexprs = get_partition_exprs(key);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+		{
+			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
+							  attnums))
+			{
+				if (used_in_expr)
+					*used_in_expr = false;
+				return true;
+			}
+		}
+		else
+		{
+			/* Arbitrary expression */
+			Node	   *expr = (Node *) lfirst(partexprs_item);
+			Bitmapset  *expr_attrs = NULL;
+
+			/* Find all attributes referenced */
+			pull_varattnos(expr, 1, &expr_attrs);
+			partexprs_item = lnext(partexprs_item);
+
+			if (bms_overlap(attnums, expr_attrs))
+			{
+				if (used_in_expr)
+					*used_in_expr = true;
+				return true;
+			}
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple
+ *		Finds partition of relation which accepts the partition key specified
+ *		in values and isnull
+ *
+ * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
+ * found or -1 if none found.
+ */
+int
+get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
+{
+	int			bound_offset;
+	int			part_index = -1;
+	PartitionKey key = RelationGetPartitionKey(relation);
+	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
+
+	/* Route as appropriate based on partitioning strategy. */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundInfo boundinfo = partdesc->boundinfo;
+				int			greatest_modulus = get_greatest_modulus(boundinfo);
+				uint64		rowHash = compute_hash_value(key, values, isnull);
+
+				part_index = boundinfo->indexes[rowHash % greatest_modulus];
+			}
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			if (isnull[0])
+			{
+				if (partition_bound_accepts_nulls(partdesc->boundinfo))
+					part_index = partdesc->boundinfo->null_index;
+			}
+			else
+			{
+				bool		equal = false;
+
+				bound_offset = partition_list_bsearch(key,
+													  partdesc->boundinfo,
+													  values[0], &equal);
+				if (bound_offset >= 0 && equal)
+					part_index = partdesc->boundinfo->indexes[bound_offset];
+			}
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				bool		equal = false,
+							range_partkey_has_null = false;
+				int			i;
+
+				/*
+				 * No range includes NULL, so this will be accepted by the
+				 * default partition if there is one, and otherwise rejected.
+				 */
+				for (i = 0; i < key->partnatts; i++)
+				{
+					if (isnull[i])
+					{
+						range_partkey_has_null = true;
+						break;
+					}
+				}
+
+				if (!range_partkey_has_null)
+				{
+					bound_offset = partition_range_datum_bsearch(key,
+														partdesc->boundinfo,
+														key->partnatts,
+														values,
+														&equal);
+					/*
+					 * The bound at bound_offset is less than or equal to the
+					 * tuple value, so the bound at offset+1 is the upper
+					 * bound of the partition we're looking for, if there
+					 * actually exists one.
+					 */
+					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
+				}
+			}
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/*
+	 * part_index < 0 means we failed to find a partition of this parent. Use
+	 * the default partition, if there is one.
+	 */
+	if (part_index < 0)
+		part_index = partdesc->boundinfo->default_index;
+
+	return part_index;
+}
+
+/*
+ * get_greatest_modulus
+ *
+ * Returns the greatest modulus of the hash partition bound. The greatest
+ * modulus will be at the end of the datums array because hash partitions are
+ * arranged in the ascending order of their modulus and remainders.
+ */
+int
+get_greatest_modulus(PartitionBoundInfo bound)
+{
+	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
+	Assert(bound->datums && bound->ndatums > 0);
+	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
+
+	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+uint64
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+	int			i;
+	int			nkeys = key->partnatts;
+	uint64		rowHash = 0;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+	for (i = 0; i < nkeys; i++)
+	{
+		if (!isnull[i])
+		{
+			Datum		hash;
+
+			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
+
+			/*
+			 * Compute hash for each datum value by calling respective
+			 * datatype-specific hash functions of each partition key
+			 * attribute.
+			 */
+			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+	if (partdesc && partdesc->boundinfo &&
+		partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
+
+/* Module-local functions. */
+
+/*
+ * generate_partition_qual
+ *
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
+ *
+ * Result expression tree is stored CacheMemoryContext to ensure it survives
+ * as long as the relcache entry. But we should be running in a less long-lived
+ * working context. To avoid leaking cache memory if this routine fails partway
+ * through, we build in working memory and then copy the completed structure
+ * into cache memory.
+ */
+static List *
+generate_partition_qual(Relation rel)
+{
+	HeapTuple	tuple;
+	MemoryContext oldcxt;
+	Datum		boundDatum;
+	bool		isnull;
+	PartitionBoundSpec *bound;
+	List	   *my_qual = NIL,
+			   *result = NIL;
+	Relation	parent;
+	bool		found_whole_row;
+
+	/* Guard against stack overflow due to overly deep partition tree */
+	check_stack_depth();
+
+	/* Quick copy */
+	if (rel->rd_partcheck != NIL)
+		return copyObject(rel->rd_partcheck);
+
+	/* Grab at least an AccessShareLock on the parent table */
+	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
+					   AccessShareLock);
+
+	/* Get pg_class.relpartbound */
+	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	boundDatum = SysCacheGetAttr(RELOID, tuple,
+								 Anum_pg_class_relpartbound,
+								 &isnull);
+	if (isnull)					/* should not happen */
+		elog(ERROR, "relation \"%s\" has relpartbound = null",
+			 RelationGetRelationName(rel));
+	bound = castNode(PartitionBoundSpec,
+					 stringToNode(TextDatumGetCString(boundDatum)));
+	ReleaseSysCache(tuple);
+
+	my_qual = get_qual_from_partbound(rel, parent, bound);
+
+	/* 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;
+
+	/*
+	 * Change Vars to have partition's attnos instead of the parent's. We do
+	 * this after we concatenate the parent's quals, because we want every Var
+	 * in it to bear this relation's attnos. It's safe to assume varno = 1
+	 * here.
+	 */
+	result = map_partition_varattnos(result, 1, rel, parent,
+									 &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");
+
+	/* Save a copy in the relcache */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	rel->rd_partcheck = copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* Keep the parent locked until commit */
+	heap_close(parent, NoLock);
+
+	return result;
+}
+
+/*
+ * partition_hbound_cmp
+ *
+ * Compares modulus first, then remainder if modulus are equal.
+ */
+static int32
+partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
+{
+	if (modulus1 < modulus2)
+		return -1;
+	if (modulus1 > modulus2)
+		return 1;
+	if (modulus1 == modulus2 && remainder1 != remainder2)
+		return (remainder1 > remainder2) ? 1 : -1;
+	return 0;
+}
+
+/*
+ * qsort_partition_hbound_cmp
+ *
+ * We sort hash bounds by modulus, then by remainder.
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+
+	return partition_hbound_cmp(h1->modulus, h1->remainder,
+								h2->modulus, h2->remainder);
+}
+
+/*
+ * qsort_partition_list_value_cmp
+ *
+ * Compare two list partition bound datums
+ */
+static int32
+qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+{
+	Datum		val1 = (*(const PartitionListValue **) a)->value,
+				val2 = (*(const PartitionListValue **) b)->value;
+	PartitionKey key = (PartitionKey) arg;
+
+	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+										   key->partcollation[0],
+										   val1, val2));
+}
+
+/*
+ * make_one_range_bound
+ *
+ * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
+ * and a flag telling whether the bound is lower or not.  Made into a function
+ * because there are multiple sites that want to use this facility.
+ */
+static PartitionRangeBound *
+make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
+{
+	PartitionRangeBound *bound;
+	ListCell   *lc;
+	int			i;
+
+	Assert(datums != NIL);
+
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
+	bound->index = index;
+	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
+	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
+													  sizeof(PartitionRangeDatumKind));
+	bound->lower = lower;
+
+	i = 0;
+	foreach(lc, datums)
+	{
+		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+
+		/* What's contained in this range datum? */
+		bound->kind[i] = datum->kind;
+
+		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		{
+			Const	   *val = castNode(Const, datum->value);
+
+			if (val->constisnull)
+				elog(ERROR, "invalid range bound datum");
+			bound->datums[i] = val->constvalue;
+		}
+
+		i++;
+	}
+
+	return bound;
+}
+
+/*
+ * partition_rbound_cmp
+ *
+ * Return for two range bounds whether the 1st one (specified in datums1,
+ * kind1, and lower1) is <, =, or > the bound specified in *b2.
+ *
+ * Note that if the values of the two range bounds compare equal, then we take
+ * into account whether they are upper or lower bounds, and an upper bound is
+ * considered to be smaller than a lower bound. This is important to the way
+ * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
+ * structure, which only stores the upper bound of a common boundary between
+ * two contiguous partitions.
+ */
+static int32
+partition_rbound_cmp(PartitionKey key,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2)
+{
+	int32		cmpval = 0;		/* placate compiler */
+	int			i;
+	Datum	   *datums2 = b2->datums;
+	PartitionRangeDatumKind *kind2 = b2->kind;
+	bool		lower2 = b2->lower;
+
+	for (i = 0; i < key->partnatts; i++)
+	{
+		/*
+		 * First, handle cases where the column is unbounded, which should not
+		 * invoke the comparison procedure, and should not consider any later
+		 * columns. Note that the PartitionRangeDatumKind enum elements
+		 * compare the same way as the values they represent.
+		 */
+		if (kind1[i] < kind2[i])
+			return -1;
+		else if (kind1[i] > kind2[i])
+			return 1;
+		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+
+			/*
+			 * The column bounds are both MINVALUE or both MAXVALUE. No later
+			 * columns should be considered, but we still need to compare
+			 * whether they are upper or lower bounds.
+			 */
+			break;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 datums1[i],
+												 datums2[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	/*
+	 * If the comparison is anything other than equal, we're done. If they
+	 * compare equal though, we still have to consider whether the boundaries
+	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
+	 * two.
+	 */
+	if (cmpval == 0 && lower1 != lower2)
+		cmpval = lower1 ? 1 : -1;
+
+	return cmpval;
+}
+
+/* Used when sorting range bounds across all range partitions */
+static int32
+qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
+{
+	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
+	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
+	PartitionKey key = (PartitionKey) arg;
+
+	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+}
+
+/*
+ * partition_list_bsearch
+ *		Returns the index of the greatest bound datum that is less than equal
+ * 		to the given value or -1 if all of the bound datums are greater
+ *
+ * *is_equal is set to true if the bound datum at the returned index is equal
+ * to the input value.
+ */
+static int
+partition_list_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+												 key->partcollation[0],
+												 boundinfo->datums[mid][0],
+												 value));
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_rbound_datum_cmp
+ *
+ * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
+ * is <, =, or > partition key of tuple (tuple_datums)
+ */
+static int32
+partition_rbound_datum_cmp(PartitionKey key,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums)
+{
+	int			i;
+	int32		cmpval = -1;
+
+	for (i = 0; i < n_tuple_datums; i++)
+	{
+		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
+			return -1;
+		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
+			return 1;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
+												 key->partcollation[i],
+												 rb_datums[i],
+												 tuple_datums[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	return cmpval;
+}
+
+/*
+ * partition_range_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given range bound or -1 if all of the range bounds are
+ *		greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input range bound
+ */
+static int
+partition_range_bsearch(PartitionKey key,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_cmp(key,
+									  boundinfo->datums[mid],
+									  boundinfo->kind[mid],
+									  (boundinfo->indexes[mid] == -1),
+									  probe);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_range_datum_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given tuple or -1 if all of the range bounds are greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input tuple.
+ */
+static int
+partition_range_datum_bsearch(PartitionKey key,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_datum_cmp(key,
+											boundinfo->datums[mid],
+											boundinfo->kind[mid],
+											values,
+											nvalues);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_hash_bsearch
+ *		Returns the index of the greatest (modulus, remainder) pair that is
+ *		less than or equal to the given (modulus, remainder) pair or -1 if
+ *		all of them are greater
+ */
+static int
+partition_hash_bsearch(PartitionKey key,
+					   PartitionBoundInfo boundinfo,
+					   int modulus, int remainder)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval,
+					bound_modulus,
+					bound_remainder;
+
+		mid = (lo + hi + 1) / 2;
+		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
+		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
+		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
+									  modulus, remainder);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+
+			if (cmpval == 0)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * get_partition_bound_num_indexes
+ *
+ * Returns the number of the entries in the partition bound indexes array.
+ */
+static int
+get_partition_bound_num_indexes(PartitionBoundInfo bound)
+{
+	int			num_indexes;
+
+	Assert(bound);
+
+	switch (bound->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * The number of the entries in the indexes array is same as the
+			 * greatest modulus.
+			 */
+			num_indexes = get_greatest_modulus(bound);
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			num_indexes = bound->ndatums;
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			/* Range partitioned table has an extra index. */
+			num_indexes = bound->ndatums + 1;
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) bound->strategy);
+	}
+
+	return num_indexes;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 1ebf9c4ed2..32fba90be7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -42,7 +42,6 @@
 #include "catalog/index.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_amproc.h"
 #include "catalog/pg_attrdef.h"
@@ -81,6 +80,7 @@
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partcache.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -261,7 +261,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
 static Relation AllocateRelationDesc(Form_pg_class relp);
 static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
 static void RelationBuildTupleDesc(Relation relation);
-static void RelationBuildPartitionKey(Relation relation);
 static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
 static void RelationInitPhysicalAddr(Relation relation);
 static void load_critical_index(Oid indexoid, Oid heapoid);
@@ -809,209 +808,6 @@ RelationBuildRuleLock(Relation relation)
 }
 
 /*
- * RelationBuildPartitionKey
- *		Build and attach to relcache partition key data of relation
- *
- * Partitioning key data is a complex structure; to avoid complicated logic to
- * free individual elements whenever the relcache entry is flushed, we give it
- * its own memory context, child of CacheMemoryContext, which can easily be
- * deleted on its own.  To avoid leaking memory in that context in case of an
- * error partway through this function, the context is initially created as a
- * child of CurTransactionContext and only re-parented to CacheMemoryContext
- * at the end, when no further errors are possible.  Also, we don't make this
- * context the current context except in very brief code sections, out of fear
- * that some of our callees allocate memory on their own which would be leaked
- * permanently.
- */
-static void
-RelationBuildPartitionKey(Relation relation)
-{
-	Form_pg_partitioned_table form;
-	HeapTuple	tuple;
-	bool		isnull;
-	int			i;
-	PartitionKey key;
-	AttrNumber *attrs;
-	oidvector  *opclass;
-	oidvector  *collation;
-	ListCell   *partexprs_item;
-	Datum		datum;
-	MemoryContext partkeycxt,
-				oldcxt;
-	int16		procnum;
-
-	tuple = SearchSysCache1(PARTRELID,
-							ObjectIdGetDatum(RelationGetRelid(relation)));
-
-	/*
-	 * The following happens when we have created our pg_class entry but not
-	 * the pg_partitioned_table entry yet.
-	 */
-	if (!HeapTupleIsValid(tuple))
-		return;
-
-	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
-											   RelationGetRelationName(relation),
-											   MEMCONTEXT_COPY_NAME,
-											   ALLOCSET_SMALL_SIZES);
-
-	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
-												sizeof(PartitionKeyData));
-
-	/* Fixed-length attributes */
-	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	key->strategy = form->partstrat;
-	key->partnatts = form->partnatts;
-
-	/*
-	 * We can rely on the first variable-length attribute being mapped to the
-	 * relevant field of the catalog's C struct, because all previous
-	 * attributes are non-nullable and fixed-length.
-	 */
-	attrs = form->partattrs.values;
-
-	/* But use the hard way to retrieve further variable-length attributes */
-	/* Operator class */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partclass, &isnull);
-	Assert(!isnull);
-	opclass = (oidvector *) DatumGetPointer(datum);
-
-	/* Collation */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partcollation, &isnull);
-	Assert(!isnull);
-	collation = (oidvector *) DatumGetPointer(datum);
-
-	/* Expressions */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partexprs, &isnull);
-	if (!isnull)
-	{
-		char	   *exprString;
-		Node	   *expr;
-
-		exprString = TextDatumGetCString(datum);
-		expr = stringToNode(exprString);
-		pfree(exprString);
-
-		/*
-		 * Run the expressions through const-simplification since the planner
-		 * will be comparing them to similarly-processed qual clause operands,
-		 * and may fail to detect valid matches without this step; fix
-		 * opfuncids while at it.  We don't need to bother with
-		 * canonicalize_qual() though, because partition expressions are not
-		 * full-fledged qualification clauses.
-		 */
-		expr = eval_const_expressions(NULL, expr);
-		fix_opfuncids(expr);
-
-		oldcxt = MemoryContextSwitchTo(partkeycxt);
-		key->partexprs = (List *) copyObject(expr);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
-	oldcxt = MemoryContextSwitchTo(partkeycxt);
-	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
-	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
-
-	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-
-	/* Gather type and collation info as well */
-	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
-	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
-	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
-	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
-	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	MemoryContextSwitchTo(oldcxt);
-
-	/* determine support function number to search for */
-	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
-		HASHEXTENDED_PROC : BTORDER_PROC;
-
-	/* Copy partattrs and fill other per-attribute info */
-	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		AttrNumber	attno = key->partattrs[i];
-		HeapTuple	opclasstup;
-		Form_pg_opclass opclassform;
-		Oid			funcid;
-
-		/* Collect opfamily information */
-		opclasstup = SearchSysCache1(CLAOID,
-									 ObjectIdGetDatum(opclass->values[i]));
-		if (!HeapTupleIsValid(opclasstup))
-			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
-
-		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
-		key->partopfamily[i] = opclassform->opcfamily;
-		key->partopcintype[i] = opclassform->opcintype;
-
-		/* Get a support function for the specified opfamily and datatypes */
-		funcid = get_opfamily_proc(opclassform->opcfamily,
-								   opclassform->opcintype,
-								   opclassform->opcintype,
-								   procnum);
-		if (!OidIsValid(funcid))
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
-							NameStr(opclassform->opcname),
-							(key->strategy == PARTITION_STRATEGY_HASH) ?
-							"hash" : "btree",
-							procnum,
-							format_type_be(opclassform->opcintype))));
-
-		fmgr_info(funcid, &key->partsupfunc[i]);
-
-		/* Collation */
-		key->partcollation[i] = collation->values[i];
-
-		/* Collect type information */
-		if (attno != 0)
-		{
-			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
-
-			key->parttypid[i] = att->atttypid;
-			key->parttypmod[i] = att->atttypmod;
-			key->parttypcoll[i] = att->attcollation;
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			key->parttypid[i] = exprType(lfirst(partexprs_item));
-			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
-			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
-
-			partexprs_item = lnext(partexprs_item);
-		}
-		get_typlenbyvalalign(key->parttypid[i],
-							 &key->parttyplen[i],
-							 &key->parttypbyval[i],
-							 &key->parttypalign[i]);
-
-		ReleaseSysCache(opclasstup);
-	}
-
-	ReleaseSysCache(tuple);
-
-	/*
-	 * Success --- reparent our context and make the relcache point to the
-	 * newly constructed key
-	 */
-	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
-	relation->rd_partkeycxt = partkeycxt;
-	relation->rd_partkey = key;
-}
-
-/*
  *		equalRuleLocks
  *
  *		Determine whether two RuleLocks are equivalent
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2faf0ca26e..21abc8babd 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -14,63 +14,23 @@
 #define PARTITION_H
 
 #include "fmgr.h"
-#include "executor/tuptable.h"
-#include "nodes/execnodes.h"
-#include "parser/parse_node.h"
+#include "nodes/parsenodes.h"
 #include "utils/rel.h"
 
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-/*
- * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
- * associated with partitioned tables as part of its partition descriptor.
- *
- * The internal structure is opaque outside partition.c.
- */
-typedef struct PartitionBoundInfoData *PartitionBoundInfo;
-
-/*
- * Information about partitions of a partitioned table.
- */
-typedef struct PartitionDescData
-{
-	int			nparts;			/* Number of partitions */
-	Oid		   *oids;			/* OIDs of partitions */
-	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
-} PartitionDescData;
-
-typedef struct PartitionDescData *PartitionDesc;
-
-extern void RelationBuildPartitionDesc(Relation relation);
-extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
-					   bool *parttypbyval, PartitionBoundInfo b1,
-					   PartitionBoundInfo b2);
-extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key);
-
-extern void check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec);
 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 fromrel_varno,
 						Relation to_rel, Relation from_rel,
 						bool *found_whole_row);
-extern List *RelationGetPartitionQual(Relation rel);
-extern Expr *get_partition_qual_relid(Oid relid);
-extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr);
 
-extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
 extern Oid	get_default_partition_oid(Oid parentId);
 extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
-extern int get_partition_for_tuple(Relation relation, Datum *values,
-						bool *isnull);
-
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 3df9c498bb..c53dfcc265 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -13,10 +13,10 @@
 #ifndef EXECPARTITION_H
 #define EXECPARTITION_H
 
-#include "catalog/partition.h"
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "utils/partcache.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
diff --git a/src/include/utils/partcache.h b/src/include/utils/partcache.h
new file mode 100644
index 0000000000..4bbd2962a9
--- /dev/null
+++ b/src/include/utils/partcache.h
@@ -0,0 +1,187 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.h
+ *		Header file for partitioning related cached data structures and
+ *		manipulation functions
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partcache.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTCACHE_H
+#define PARTCACHE_H
+
+#include "fmgr.h"
+#include "utils/relcache.h"
+
+/*
+ * Information about the partition key of a relation
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partitioning strategy */
+	int16		partnatts;		/* number of columns in the partition key */
+	AttrNumber *partattrs;		/* attribute numbers of columns in the
+								 * partition key */
+	List	   *partexprs;		/* list of expressions in the partitioning
+								 * key, or NIL */
+
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+
+	/* Partitioning collation per attribute */
+	Oid		   *partcollation;
+
+	/* Type information per attribute */
+	Oid		   *parttypid;
+	int32	   *parttypmod;
+	int16	   *parttyplen;
+	bool	   *parttypbyval;
+	char	   *parttypalign;
+	Oid		   *parttypcoll;
+}			PartitionKeyData;
+
+typedef struct PartitionKeyData *PartitionKey;
+
+typedef struct PartitionBoundInfoData *PartitionBoundInfo;
+
+/*
+ * Information about partitions of a partitioned table.
+ */
+typedef struct PartitionDescData
+{
+	int			nparts;			/* Number of partitions */
+	Oid		   *oids;			/* OIDs of partitions */
+	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
+} PartitionDescData;
+
+typedef struct PartitionDescData *PartitionDesc;
+
+/*
+ * Information about bounds of a partitioned relation
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+	char		strategy;		/* hash, list or range? */
+	int			ndatums;		/* Length of the datums following array */
+	Datum	  **datums;
+	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+									 * NULL for hash and list partitioned
+									 * tables */
+	int		   *indexes;		/* Partition indexes */
+	int			null_index;		/* Index of the null-accepting partition; -1
+								 * if there isn't one */
+	int			default_index;	/* Index of the default partition; -1 if there
+								 * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * PartitionKey inquiry functions
+ */
+static inline int
+get_partition_strategy(PartitionKey key)
+{
+	return key->strategy;
+}
+
+static inline int
+get_partition_natts(PartitionKey key)
+{
+	return key->partnatts;
+}
+
+static inline List *
+get_partition_exprs(PartitionKey key)
+{
+	return key->partexprs;
+}
+
+/*
+ * PartitionKey inquiry functions - one column
+ */
+static inline int16
+get_partition_col_attnum(PartitionKey key, int col)
+{
+	return key->partattrs[col];
+}
+
+static inline Oid
+get_partition_col_typid(PartitionKey key, int col)
+{
+	return key->parttypid[col];
+}
+
+static inline int32
+get_partition_col_typmod(PartitionKey key, int col)
+{
+	return key->parttypmod[col];
+}
+
+extern void RelationBuildPartitionKey(Relation relation);
+extern void RelationBuildPartitionDesc(Relation relation);
+extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
+					   bool *parttypbyval, PartitionBoundInfo b1,
+					   PartitionBoundInfo b2);
+
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
+
+extern void check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec);
+
+extern List *RelationGetPartitionQual(Relation rel);
+extern Expr *get_partition_qual_relid(Oid relid);
+
+extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr);
+
+extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+
+extern int get_greatest_modulus(PartitionBoundInfo b);
+extern uint64 compute_hash_value(PartitionKey key, Datum *values,
+						bool *isnull);
+
+/* For tuple routing */
+extern int get_partition_for_tuple(Relation relation, Datum *values,
+						bool *isnull);
+
+#endif							/* PARTCACHE_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index aa8add544a..db3545ec7f 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -19,12 +19,11 @@
 #include "catalog/pg_class.h"
 #include "catalog/pg_index.h"
 #include "catalog/pg_publication.h"
-#include "fmgr.h"
 #include "nodes/bitmapset.h"
 #include "rewrite/prs2lock.h"
 #include "storage/block.h"
 #include "storage/relfilenode.h"
-#include "utils/relcache.h"
+#include "utils/partcache.h"
 #include "utils/reltrigger.h"
 
 
@@ -47,36 +46,6 @@ typedef struct LockInfoData
 typedef LockInfoData *LockInfo;
 
 /*
- * Information about the partition key of a relation
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partitioning strategy */
-	int16		partnatts;		/* number of columns in the partition key */
-	AttrNumber *partattrs;		/* attribute numbers of columns in the
-								 * partition key */
-	List	   *partexprs;		/* list of expressions in the partitioning
-								 * key, or NIL */
-
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-
-	/* Partitioning collation per attribute */
-	Oid		   *partcollation;
-
-	/* Type information per attribute */
-	Oid		   *parttypid;
-	int32	   *parttypmod;
-	int16	   *parttyplen;
-	bool	   *parttypbyval;
-	char	   *parttypalign;
-	Oid		   *parttypcoll;
-}			PartitionKeyData;
-
-typedef struct PartitionKeyData *PartitionKey;
-
-/*
  * Here are the contents of a relation cache entry.
  */
 
@@ -593,48 +562,6 @@ typedef struct ViewOptions
 #define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
 
 /*
- * PartitionKey inquiry functions
- */
-static inline int
-get_partition_strategy(PartitionKey key)
-{
-	return key->strategy;
-}
-
-static inline int
-get_partition_natts(PartitionKey key)
-{
-	return key->partnatts;
-}
-
-static inline List *
-get_partition_exprs(PartitionKey key)
-{
-	return key->partexprs;
-}
-
-/*
- * PartitionKey inquiry functions - one column
- */
-static inline int16
-get_partition_col_attnum(PartitionKey key, int col)
-{
-	return key->partattrs[col];
-}
-
-static inline Oid
-get_partition_col_typid(PartitionKey key, int col)
-{
-	return key->parttypid[col];
-}
-
-static inline int32
-get_partition_col_typmod(PartitionKey key, int col)
-{
-	return key->parttypmod[col];
-}
-
-/*
  * RelationGetPartitionDesc
  *		Returns partition descriptor for a relation.
  */
-- 
2.11.0

#11David Steele
david@pgmasters.net
In reply to: Amit Langote (#10)
Re: Re: reorganizing partitioning code

Hi Amit,

On 2/16/18 3:36 AM, Amit Langote wrote:

Attached updated version.

This patch no longer applies and the conflicts do not appear to be trivial.

I'm a bit confused about your comment in [1]/messages/by-id/33098109-9ef1-9594-e7d5-0977a50f8cfa@lab.ntt.co.jp:

I gave up on rebasing this patch yesterday as I couldn't finish it in
5 minutes, but maybe I will try later this month. Gotta focus on
thefaster pruning stuff for now...

How much later are we talking about?

Marked Waiting on Author.

--
-David
david@pgmasters.net

[1]: /messages/by-id/33098109-9ef1-9594-e7d5-0977a50f8cfa@lab.ntt.co.jp
/messages/by-id/33098109-9ef1-9594-e7d5-0977a50f8cfa@lab.ntt.co.jp

#12Amit Langote
amitlangote09@gmail.com
In reply to: David Steele (#11)
Re: Re: reorganizing partitioning code

Hi David.

On Fri, Mar 2, 2018 at 11:53 PM, David Steele <david@pgmasters.net> wrote:

Hi Amit,

On 2/16/18 3:36 AM, Amit Langote wrote:

Attached updated version.

This patch no longer applies and the conflicts do not appear to be trivial.

I'm a bit confused about your comment in [1]:

I gave up on rebasing this patch yesterday as I couldn't finish it in
5 minutes, but maybe I will try later this month. Gotta focus on
thefaster pruning stuff for now...

How much later are we talking about?

Sorry about the confusing comment. It could be sometime later half of
the next week.

Thanks,
Amit

#13David Steele
david@pgmasters.net
In reply to: Amit Langote (#12)
Re: Re: Re: reorganizing partitioning code

Hi Amit,

On 3/2/18 11:17 AM, Amit Langote wrote:

On Fri, Mar 2, 2018 at 11:53 PM, David Steele <david@pgmasters.net> wrote:

Hi Amit,

On 2/16/18 3:36 AM, Amit Langote wrote:

Attached updated version.

This patch no longer applies and the conflicts do not appear to be trivial.

I'm a bit confused about your comment in [1]:

I gave up on rebasing this patch yesterday as I couldn't finish it in
5 minutes, but maybe I will try later this month. Gotta focus on
thefaster pruning stuff for now...

How much later are we talking about?

Sorry about the confusing comment. It could be sometime later half of
the next week.

We are now three weeks into the CF with no new patch.

Are you planning to update this patch? If not, I think it should be
marked as Returned with Feedback and submitted to the next CF once it
has been updated.

Regards,
--
-David
david@pgmasters.net

#14Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Steele (#13)
Re: Re: Re: reorganizing partitioning code

David Steele wrote:

Sorry about the confusing comment. It could be sometime later half of
the next week.

We are now three weeks into the CF with no new patch.

Are you planning to update this patch? If not, I think it should be
marked as Returned with Feedback and submitted to the next CF once it
has been updated.

This is no new development, only code movement. I think it would be
worse to have three different branches of partitioning code, v10
"basic", v11 "powerful but not reorganized", v12 "reorganized". I'd
rather have only v10 "basic" and v11+ "powerful".

Let's keep this entry open till the last minute.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#15Tom Lane
tgl@sss.pgh.pa.us
In reply to: Alvaro Herrera (#14)
Re: reorganizing partitioning code

Alvaro Herrera <alvherre@alvh.no-ip.org> writes:

David Steele wrote:

Are you planning to update this patch? If not, I think it should be
marked as Returned with Feedback and submitted to the next CF once it
has been updated.

This is no new development, only code movement. I think it would be
worse to have three different branches of partitioning code, v10
"basic", v11 "powerful but not reorganized", v12 "reorganized". I'd
rather have only v10 "basic" and v11+ "powerful".

Let's keep this entry open till the last minute.

Nonetheless, it's March 21. David's point is that it's time to get a
move on.

regards, tom lane

#16Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Tom Lane (#15)
1 attachment(s)
Re: reorganizing partitioning code

On 2018/03/22 2:33, Tom Lane wrote:

Alvaro Herrera <alvherre@alvh.no-ip.org> writes:

David Steele wrote:

Are you planning to update this patch? If not, I think it should be
marked as Returned with Feedback and submitted to the next CF once it
has been updated.

This is no new development, only code movement. I think it would be
worse to have three different branches of partitioning code, v10
"basic", v11 "powerful but not reorganized", v12 "reorganized". I'd
rather have only v10 "basic" and v11+ "powerful".

Let's keep this entry open till the last minute.

Nonetheless, it's March 21. David's point is that it's time to get a
move on.

I'm sorry it took me a while to reply on this thread.

Due to quite a few changes to the partitioning-related code recently and
also considering some pending patches which might touch the code moved
around by this patch, I'd been putting off rebasing this patch. Although,
I should have said that before without waiting until today to do so. Sorry.

FWIW, I did manage to rebase it this morning and posting it here.

Thanks,
Amit

Attachments:

v5-0001-Reorganize-partitioning-code.patchtext/plain; charset=UTF-8; name=v5-0001-Reorganize-partitioning-code.patchDownload
From add2110378b762a7118132fc400c977e10b782a9 Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Tue, 13 Feb 2018 15:59:30 +0900
Subject: [PATCH v5] Reorganize partitioning code

---
 src/backend/catalog/partition.c        | 1957 +----------------------------
 src/backend/executor/execMain.c        |    1 -
 src/backend/executor/execPartition.c   |    1 +
 src/backend/optimizer/path/joinrels.c  |    2 +-
 src/backend/optimizer/prep/prepunion.c |    1 -
 src/backend/optimizer/util/plancat.c   |    1 -
 src/backend/optimizer/util/relnode.c   |    2 +-
 src/backend/utils/adt/ruleutils.c      |    1 -
 src/backend/utils/cache/Makefile       |    6 +-
 src/backend/utils/cache/partcache.c    | 2145 ++++++++++++++++++++++++++++++++
 src/backend/utils/cache/relcache.c     |  207 +--
 src/include/catalog/partition.h        |   42 +-
 src/include/executor/execPartition.h   |    2 +-
 src/include/utils/partcache.h          |  187 +++
 src/include/utils/rel.h                |   75 +-
 15 files changed, 2343 insertions(+), 2287 deletions(-)
 create mode 100644 src/backend/utils/cache/partcache.c
 create mode 100644 src/include/utils/partcache.h

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 53855f5088..a9257b0cd8 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,11 +15,7 @@
 
 #include "postgres.h"
 
-#include "access/hash.h"
-#include "access/heapam.h"
 #include "access/htup_details.h"
-#include "access/nbtree.h"
-#include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
@@ -35,7 +31,6 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
-#include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
 #include "optimizer/prep.h"
@@ -48,102 +43,12 @@
 #include "utils/datum.h"
 #include "utils/fmgroids.h"
 #include "utils/hashutils.h"
-#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
-#include "utils/ruleutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-	char		strategy;		/* hash, list or range? */
-	int			ndatums;		/* Length of the datums following array */
-	Datum	  **datums;
-	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-									 * NULL for hash and list partitioned
-									 * tables */
-	int		   *indexes;		/* Partition indexes */
-	int			null_index;		/* Index of the null-accepting partition; -1
-								 * if there isn't one */
-	int			default_index;	/* Index of the default partition; -1 if there
-								 * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-	int			modulus;
-	int			remainder;
-	int			index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-	int			index;
-	Datum		value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-	int			index;
-	Datum	   *datums;			/* range bound datums */
-	PartitionRangeDatumKind *kind;	/* the kind of each datum */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
-static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
-static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
-							   void *arg);
-static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
-						   void *arg);
-
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
@@ -159,1070 +64,6 @@ static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 				   bool for_default);
 static List *get_range_nulltest(PartitionKey key);
-static List *generate_partition_qual(Relation rel);
-
-static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
-					 List *datums, bool lower);
-static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
-					 int remainder2);
-static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
-					 Oid *partcollation, Datum *datums1,
-					 PartitionRangeDatumKind *kind1, bool lower1,
-					 PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
-						   Oid *partcollation,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal);
-static int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
-						Oid *partcollation,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
-							  Oid *partcollation,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionBoundInfo boundinfo,
-					   int modulus, int remainder);
-
-static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int	get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
-								 Datum *values, bool *isnull);
-
-/*
- * RelationBuildPartitionDesc
- *		Form rel's partition descriptor
- *
- * Not flushed from the cache by RelationClearRelation() unless changed because
- * of addition or removal of partition.
- */
-void
-RelationBuildPartitionDesc(Relation rel)
-{
-	List	   *inhoids,
-			   *partoids;
-	Oid		   *oids = NULL;
-	List	   *boundspecs = NIL;
-	ListCell   *cell;
-	int			i,
-				nparts;
-	PartitionKey key = RelationGetPartitionKey(rel);
-	PartitionDesc result;
-	MemoryContext oldcxt;
-
-	int			ndatums = 0;
-	int			default_index = -1;
-
-	/* Hash partitioning specific */
-	PartitionHashBound **hbounds = NULL;
-
-	/* List partitioning specific */
-	PartitionListValue **all_values = NULL;
-	int			null_index = -1;
-
-	/* Range partitioning specific */
-	PartitionRangeBound **rbounds = NULL;
-
-	/* Get partition oids from pg_inherits */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
-
-	/* Collect bound spec nodes in a list */
-	i = 0;
-	partoids = NIL;
-	foreach(cell, inhoids)
-	{
-		Oid			inhrelid = lfirst_oid(cell);
-		HeapTuple	tuple;
-		Datum		datum;
-		bool		isnull;
-		Node	   *boundspec;
-
-		tuple = SearchSysCache1(RELOID, inhrelid);
-		if (!HeapTupleIsValid(tuple))
-			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
-
-		/*
-		 * It is possible that the pg_class tuple of a partition has not been
-		 * updated yet to set its relpartbound field.  The only case where
-		 * this happens is when we open the parent relation to check using its
-		 * partition descriptor that a new partition's bound does not overlap
-		 * some existing partition.
-		 */
-		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
-		{
-			ReleaseSysCache(tuple);
-			continue;
-		}
-
-		datum = SysCacheGetAttr(RELOID, tuple,
-								Anum_pg_class_relpartbound,
-								&isnull);
-		Assert(!isnull);
-		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
-
-		/*
-		 * Sanity check: If the PartitionBoundSpec says this is the default
-		 * partition, its OID should correspond to whatever's stored in
-		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
-		 */
-		if (castNode(PartitionBoundSpec, boundspec)->is_default)
-		{
-			Oid			partdefid;
-
-			partdefid = get_default_partition_oid(RelationGetRelid(rel));
-			if (partdefid != inhrelid)
-				elog(ERROR, "expected partdefid %u, but got %u",
-					 inhrelid, partdefid);
-		}
-
-		boundspecs = lappend(boundspecs, boundspec);
-		partoids = lappend_oid(partoids, inhrelid);
-		ReleaseSysCache(tuple);
-	}
-
-	nparts = list_length(partoids);
-
-	if (nparts > 0)
-	{
-		oids = (Oid *) palloc(nparts * sizeof(Oid));
-		i = 0;
-		foreach(cell, partoids)
-			oids[i++] = lfirst_oid(cell);
-
-		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_HASH)
-		{
-			ndatums = nparts;
-			hbounds = (PartitionHashBound **)
-				palloc(nparts * sizeof(PartitionHashBound *));
-
-			i = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-
-				if (spec->strategy != PARTITION_STRATEGY_HASH)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				hbounds[i] = (PartitionHashBound *)
-					palloc(sizeof(PartitionHashBound));
-
-				hbounds[i]->modulus = spec->modulus;
-				hbounds[i]->remainder = spec->remainder;
-				hbounds[i]->index = i;
-				i++;
-			}
-
-			/* Sort all the bounds in ascending order */
-			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
-				  qsort_partition_hbound_cmp);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_LIST)
-		{
-			List	   *non_null_values = NIL;
-
-			/*
-			 * Create a unified list of non-null values across all partitions.
-			 */
-			i = 0;
-			null_index = -1;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				ListCell   *c;
-
-				if (spec->strategy != PARTITION_STRATEGY_LIST)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the list of non-null
-				 * datums for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i;
-					i++;
-					continue;
-				}
-
-				foreach(c, spec->listdatums)
-				{
-					Const	   *val = castNode(Const, lfirst(c));
-					PartitionListValue *list_value = NULL;
-
-					if (!val->constisnull)
-					{
-						list_value = (PartitionListValue *)
-							palloc0(sizeof(PartitionListValue));
-						list_value->index = i;
-						list_value->value = val->constvalue;
-					}
-					else
-					{
-						/*
-						 * Never put a null into the values array, flag
-						 * instead for the code further down below where we
-						 * construct the actual relcache struct.
-						 */
-						if (null_index != -1)
-							elog(ERROR, "found null more than once");
-						null_index = i;
-					}
-
-					if (list_value)
-						non_null_values = lappend(non_null_values,
-												  list_value);
-				}
-
-				i++;
-			}
-
-			ndatums = list_length(non_null_values);
-
-			/*
-			 * Collect all list values in one array. Alongside the value, we
-			 * also save the index of partition the value comes from.
-			 */
-			all_values = (PartitionListValue **) palloc(ndatums *
-														sizeof(PartitionListValue *));
-			i = 0;
-			foreach(cell, non_null_values)
-			{
-				PartitionListValue *src = lfirst(cell);
-
-				all_values[i] = (PartitionListValue *)
-					palloc(sizeof(PartitionListValue));
-				all_values[i]->value = src->value;
-				all_values[i]->index = src->index;
-				i++;
-			}
-
-			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
-					  qsort_partition_list_value_cmp, (void *) key);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_RANGE)
-		{
-			int			k;
-			PartitionRangeBound **all_bounds,
-					   *prev;
-
-			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
-														  sizeof(PartitionRangeBound *));
-
-			/*
-			 * Create a unified list of range bounds across all the
-			 * partitions.
-			 */
-			i = ndatums = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				PartitionRangeBound *lower,
-						   *upper;
-
-				if (spec->strategy != PARTITION_STRATEGY_RANGE)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the allbounds array
-				 * for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i++;
-					continue;
-				}
-
-				lower = make_one_range_bound(key, i, spec->lowerdatums,
-											 true);
-				upper = make_one_range_bound(key, i, spec->upperdatums,
-											 false);
-				all_bounds[ndatums++] = lower;
-				all_bounds[ndatums++] = upper;
-				i++;
-			}
-
-			Assert(ndatums == nparts * 2 ||
-				   (default_index != -1 && ndatums == (nparts - 1) * 2));
-
-			/* Sort all the bounds in ascending order */
-			qsort_arg(all_bounds, ndatums,
-					  sizeof(PartitionRangeBound *),
-					  qsort_partition_rbound_cmp,
-					  (void *) key);
-
-			/* Save distinct bounds from all_bounds into rbounds. */
-			rbounds = (PartitionRangeBound **)
-				palloc(ndatums * sizeof(PartitionRangeBound *));
-			k = 0;
-			prev = NULL;
-			for (i = 0; i < ndatums; i++)
-			{
-				PartitionRangeBound *cur = all_bounds[i];
-				bool		is_distinct = false;
-				int			j;
-
-				/* Is the current bound distinct from the previous one? */
-				for (j = 0; j < key->partnatts; j++)
-				{
-					Datum		cmpval;
-
-					if (prev == NULL || cur->kind[j] != prev->kind[j])
-					{
-						is_distinct = true;
-						break;
-					}
-
-					/*
-					 * If the bounds are both MINVALUE or MAXVALUE, stop now
-					 * and treat them as equal, since any values after this
-					 * point must be ignored.
-					 */
-					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
-						break;
-
-					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
-											   key->partcollation[j],
-											   cur->datums[j],
-											   prev->datums[j]);
-					if (DatumGetInt32(cmpval) != 0)
-					{
-						is_distinct = true;
-						break;
-					}
-				}
-
-				/*
-				 * Only if the bound is distinct save it into a temporary
-				 * array i.e. rbounds which is later copied into boundinfo
-				 * datums array.
-				 */
-				if (is_distinct)
-					rbounds[k++] = all_bounds[i];
-
-				prev = cur;
-			}
-
-			/* Update ndatums to hold the count of distinct datums. */
-			ndatums = k;
-		}
-		else
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/* Now build the actual relcache partition descriptor */
-	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
-												  RelationGetRelationName(rel),
-												  MEMCONTEXT_COPY_NAME,
-												  ALLOCSET_DEFAULT_SIZES);
-	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
-
-	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
-	result->nparts = nparts;
-	if (nparts > 0)
-	{
-		PartitionBoundInfo boundinfo;
-		int		   *mapping;
-		int			next_index = 0;
-
-		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
-
-		boundinfo = (PartitionBoundInfoData *)
-			palloc0(sizeof(PartitionBoundInfoData));
-		boundinfo->strategy = key->strategy;
-		boundinfo->default_index = -1;
-		boundinfo->ndatums = ndatums;
-		boundinfo->null_index = -1;
-		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
-
-		/* Initialize mapping array with invalid values */
-		mapping = (int *) palloc(sizeof(int) * nparts);
-		for (i = 0; i < nparts; i++)
-			mapping[i] = -1;
-
-		switch (key->strategy)
-		{
-			case PARTITION_STRATEGY_HASH:
-				{
-					/* Modulus are stored in ascending order */
-					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
-
-					boundinfo->indexes = (int *) palloc(greatest_modulus *
-														sizeof(int));
-
-					for (i = 0; i < greatest_modulus; i++)
-						boundinfo->indexes[i] = -1;
-
-					for (i = 0; i < nparts; i++)
-					{
-						int			modulus = hbounds[i]->modulus;
-						int			remainder = hbounds[i]->remainder;
-
-						boundinfo->datums[i] = (Datum *) palloc(2 *
-																sizeof(Datum));
-						boundinfo->datums[i][0] = Int32GetDatum(modulus);
-						boundinfo->datums[i][1] = Int32GetDatum(remainder);
-
-						while (remainder < greatest_modulus)
-						{
-							/* overlap? */
-							Assert(boundinfo->indexes[remainder] == -1);
-							boundinfo->indexes[remainder] = i;
-							remainder += modulus;
-						}
-
-						mapping[hbounds[i]->index] = i;
-						pfree(hbounds[i]);
-					}
-					pfree(hbounds);
-					break;
-				}
-
-			case PARTITION_STRATEGY_LIST:
-				{
-					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
-
-					/*
-					 * Copy values.  Indexes of individual values are mapped
-					 * to canonical values so that they match for any two list
-					 * partitioned tables with same number of partitions and
-					 * same lists per partition.  One way to canonicalize is
-					 * to assign the index in all_values[] of the smallest
-					 * value of each partition, as the index of all of the
-					 * partition's values.
-					 */
-					for (i = 0; i < ndatums; i++)
-					{
-						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
-						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
-															key->parttypbyval[0],
-															key->parttyplen[0]);
-
-						/* If the old index has no mapping, assign one */
-						if (mapping[all_values[i]->index] == -1)
-							mapping[all_values[i]->index] = next_index++;
-
-						boundinfo->indexes[i] = mapping[all_values[i]->index];
-					}
-
-					/*
-					 * If null-accepting partition has no mapped index yet,
-					 * assign one.  This could happen if such partition
-					 * accepts only null and hence not covered in the above
-					 * loop which only handled non-null values.
-					 */
-					if (null_index != -1)
-					{
-						Assert(null_index >= 0);
-						if (mapping[null_index] == -1)
-							mapping[null_index] = next_index++;
-						boundinfo->null_index = mapping[null_index];
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						/*
-						 * The default partition accepts any value not
-						 * specified in the lists of other partitions, hence
-						 * it should not get mapped index while assigning
-						 * those for non-null datums.
-						 */
-						Assert(default_index >= 0 &&
-							   mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-
-					/* All partition must now have a valid mapping */
-					Assert(next_index == nparts);
-					break;
-				}
-
-			case PARTITION_STRATEGY_RANGE:
-				{
-					boundinfo->kind = (PartitionRangeDatumKind **)
-						palloc(ndatums *
-							   sizeof(PartitionRangeDatumKind *));
-					boundinfo->indexes = (int *) palloc((ndatums + 1) *
-														sizeof(int));
-
-					for (i = 0; i < ndatums; i++)
-					{
-						int			j;
-
-						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
-																sizeof(Datum));
-						boundinfo->kind[i] = (PartitionRangeDatumKind *)
-							palloc(key->partnatts *
-								   sizeof(PartitionRangeDatumKind));
-						for (j = 0; j < key->partnatts; j++)
-						{
-							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
-								boundinfo->datums[i][j] =
-									datumCopy(rbounds[i]->datums[j],
-											  key->parttypbyval[j],
-											  key->parttyplen[j]);
-							boundinfo->kind[i][j] = rbounds[i]->kind[j];
-						}
-
-						/*
-						 * There is no mapping for invalid indexes.
-						 *
-						 * Any lower bounds in the rbounds array have invalid
-						 * indexes assigned, because the values between the
-						 * previous bound (if there is one) and this (lower)
-						 * bound are not part of the range of any existing
-						 * partition.
-						 */
-						if (rbounds[i]->lower)
-							boundinfo->indexes[i] = -1;
-						else
-						{
-							int			orig_index = rbounds[i]->index;
-
-							/* If the old index has no mapping, assign one */
-							if (mapping[orig_index] == -1)
-								mapping[orig_index] = next_index++;
-
-							boundinfo->indexes[i] = mapping[orig_index];
-						}
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						Assert(default_index >= 0 && mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-					boundinfo->indexes[i] = -1;
-					break;
-				}
-
-			default:
-				elog(ERROR, "unexpected partition strategy: %d",
-					 (int) key->strategy);
-		}
-
-		result->boundinfo = boundinfo;
-
-		/*
-		 * Now assign OIDs from the original array into mapped indexes of the
-		 * result array.  Order of OIDs in the former is defined by the
-		 * catalog scan that retrieved them, whereas that in the latter is
-		 * defined by canonicalized representation of the partition bounds.
-		 */
-		for (i = 0; i < nparts; i++)
-			result->oids[mapping[i]] = oids[i];
-		pfree(mapping);
-	}
-
-	MemoryContextSwitchTo(oldcxt);
-	rel->rd_partdesc = result;
-}
-
-/*
- * Are two partition bound collections logically equal?
- *
- * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
- * This is also useful when b1 and b2 are bound collections of two separate
- * relations, respectively, because PartitionBoundInfo is a canonical
- * representation of partition bounds.
- */
-bool
-partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
-					   PartitionBoundInfo b1, PartitionBoundInfo b2)
-{
-	int			i;
-
-	if (b1->strategy != b2->strategy)
-		return false;
-
-	if (b1->ndatums != b2->ndatums)
-		return false;
-
-	if (b1->null_index != b2->null_index)
-		return false;
-
-	if (b1->default_index != b2->default_index)
-		return false;
-
-	if (b1->strategy == PARTITION_STRATEGY_HASH)
-	{
-		int			greatest_modulus = get_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_greatest_modulus(b2))
-			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their modulus
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
-	}
-	else
-	{
-		for (i = 0; i < b1->ndatums; i++)
-		{
-			int			j;
-
-			for (j = 0; j < partnatts; j++)
-			{
-				/* For range partitions, the bounds might not be finite. */
-				if (b1->kind != NULL)
-				{
-					/* The different kinds of bound all differ from each other */
-					if (b1->kind[i][j] != b2->kind[i][j])
-						return false;
-
-					/*
-					 * Non-finite bounds are equal without further
-					 * examination.
-					 */
-					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
-						continue;
-				}
-
-				/*
-				 * Compare the actual values. Note that it would be both
-				 * incorrect and unsafe to invoke the comparison operator
-				 * derived from the partitioning specification here.  It would
-				 * be incorrect because we want the relcache entry to be
-				 * updated for ANY change to the partition bounds, not just
-				 * those that the partitioning operator thinks are
-				 * significant.  It would be unsafe because we might reach
-				 * this code in the context of an aborted transaction, and an
-				 * arbitrary partitioning operator might not be safe in that
-				 * context.  datumIsEqual() should be simple enough to be
-				 * safe.
-				 */
-				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-								  parttypbyval[j], parttyplen[j]))
-					return false;
-			}
-
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-		}
-
-		/* There are ndatums+1 indexes in case of range partitions */
-		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
-			b1->indexes[i] != b2->indexes[i])
-			return false;
-	}
-	return true;
-}
-
-/*
- * Return a copy of given PartitionBoundInfo structure. The data types of bounds
- * are described by given partition key specification.
- */
-PartitionBoundInfo
-partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key)
-{
-	PartitionBoundInfo dest;
-	int			i;
-	int			ndatums;
-	int			partnatts;
-	int			num_indexes;
-
-	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
-
-	dest->strategy = src->strategy;
-	ndatums = dest->ndatums = src->ndatums;
-	partnatts = key->partnatts;
-
-	num_indexes = get_partition_bound_num_indexes(src);
-
-	/* List partitioned tables have only a single partition key. */
-	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
-
-	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
-
-	if (src->kind != NULL)
-	{
-		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
-														 sizeof(PartitionRangeDatumKind *));
-		for (i = 0; i < ndatums; i++)
-		{
-			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
-															   sizeof(PartitionRangeDatumKind));
-
-			memcpy(dest->kind[i], src->kind[i],
-				   sizeof(PartitionRangeDatumKind) * key->partnatts);
-		}
-	}
-	else
-		dest->kind = NULL;
-
-	for (i = 0; i < ndatums; i++)
-	{
-		int			j;
-
-		/*
-		 * For a corresponding to hash partition, datums array will have two
-		 * elements - modulus and remainder.
-		 */
-		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
-		int			natts = hash_part ? 2 : partnatts;
-
-		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
-
-		for (j = 0; j < natts; j++)
-		{
-			bool		byval;
-			int			typlen;
-
-			if (hash_part)
-			{
-				typlen = sizeof(int32); /* Always int4 */
-				byval = true;	/* int4 is pass-by-value */
-			}
-			else
-			{
-				byval = key->parttypbyval[j];
-				typlen = key->parttyplen[j];
-			}
-
-			if (dest->kind == NULL ||
-				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
-				dest->datums[i][j] = datumCopy(src->datums[i][j],
-											   byval, typlen);
-		}
-	}
-
-	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
-	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
-
-	dest->null_index = src->null_index;
-	dest->default_index = src->default_index;
-
-	return dest;
-}
-
-/*
- * check_new_partition_bound
- *
- * Checks if the new partition's bound overlaps any of the existing partitions
- * of parent.  Also performs additional checks as necessary per strategy.
- */
-void
-check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	PartitionBoundInfo boundinfo = partdesc->boundinfo;
-	ParseState *pstate = make_parsestate(NULL);
-	int			with = -1;
-	bool		overlap = false;
-
-	if (spec->is_default)
-	{
-		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
-			return;
-
-		/* Default partition already exists, error out. */
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
-				 parser_errposition(pstate, spec->location)));
-	}
-
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					Datum	  **datums = boundinfo->datums;
-					int			ndatums = boundinfo->ndatums;
-					int			greatest_modulus;
-					int			remainder;
-					int			offset;
-					bool		valid_modulus = true;
-					int			prev_modulus,	/* Previous largest modulus */
-								next_modulus;	/* Next largest modulus */
-
-					/*
-					 * Check rule that every modulus must be a factor of the
-					 * next larger modulus.  For example, if you have a bunch
-					 * of partitions that all have modulus 5, you can add a
-					 * new partition with modulus 10 or a new partition with
-					 * modulus 15, but you cannot add both a partition with
-					 * modulus 10 and a partition with modulus 15, because 10
-					 * is not a factor of 15.
-					 *
-					 * Get the greatest (modulus, remainder) pair contained in
-					 * boundinfo->datums that is less than or equal to the
-					 * (spec->modulus, spec->remainder) pair.
-					 */
-					offset = partition_hash_bsearch(boundinfo,
-													spec->modulus,
-													spec->remainder);
-					if (offset < 0)
-					{
-						next_modulus = DatumGetInt32(datums[0][0]);
-						valid_modulus = (next_modulus % spec->modulus) == 0;
-					}
-					else
-					{
-						prev_modulus = DatumGetInt32(datums[offset][0]);
-						valid_modulus = (spec->modulus % prev_modulus) == 0;
-
-						if (valid_modulus && (offset + 1) < ndatums)
-						{
-							next_modulus = DatumGetInt32(datums[offset + 1][0]);
-							valid_modulus = (next_modulus % spec->modulus) == 0;
-						}
-					}
-
-					if (!valid_modulus)
-						ereport(ERROR,
-								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
-
-					greatest_modulus = get_greatest_modulus(boundinfo);
-					remainder = spec->remainder;
-
-					/*
-					 * Normally, the lowest remainder that could conflict with
-					 * the new partition is equal to the remainder specified
-					 * for the new partition, but when the new partition has a
-					 * modulus higher than any used so far, we need to adjust.
-					 */
-					if (remainder >= greatest_modulus)
-						remainder = remainder % greatest_modulus;
-
-					/* Check every potentially-conflicting remainder. */
-					do
-					{
-						if (boundinfo->indexes[remainder] != -1)
-						{
-							overlap = true;
-							with = boundinfo->indexes[remainder];
-							break;
-						}
-						remainder += spec->modulus;
-					} while (remainder < greatest_modulus);
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_LIST:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-
-				if (partdesc->nparts > 0)
-				{
-					ListCell   *cell;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo) ||
-							partition_bound_has_default(boundinfo)));
-
-					foreach(cell, spec->listdatums)
-					{
-						Const	   *val = castNode(Const, lfirst(cell));
-
-						if (!val->constisnull)
-						{
-							int			offset;
-							bool		equal;
-
-							offset = partition_list_bsearch(key->partsupfunc,
-														key->partcollation,
-															boundinfo,
-															val->constvalue,
-															&equal);
-							if (offset >= 0 && equal)
-							{
-								overlap = true;
-								with = boundinfo->indexes[offset];
-								break;
-							}
-						}
-						else if (partition_bound_accepts_nulls(boundinfo))
-						{
-							overlap = true;
-							with = boundinfo->null_index;
-							break;
-						}
-					}
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				PartitionRangeBound *lower,
-						   *upper;
-
-				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
-				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
-
-				/*
-				 * First check if the resulting range would be empty with
-				 * specified lower and upper bounds
-				 */
-				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
-										 key->partcollation, lower->datums,
-										 lower->kind, true, upper) >= 0)
-				{
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-							 errmsg("empty range bound specified for partition \"%s\"",
-									relname),
-							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
-									   get_range_partbound_string(spec->lowerdatums),
-									   get_range_partbound_string(spec->upperdatums)),
-							 parser_errposition(pstate, spec->location)));
-				}
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					int			offset;
-					bool		equal;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_has_default(boundinfo)));
-
-					/*
-					 * Test whether the new lower bound (which is treated
-					 * inclusively as part of the new partition) lies inside
-					 * an existing partition, or in a gap.
-					 *
-					 * If it's inside an existing partition, the bound at
-					 * offset + 1 will be the upper bound of that partition,
-					 * and its index will be >= 0.
-					 *
-					 * If it's in a gap, the bound at offset + 1 will be the
-					 * lower bound of the next partition, and its index will
-					 * be -1. This is also true if there is no next partition,
-					 * since the index array is initialised with an extra -1
-					 * at the end.
-					 */
-					offset = partition_range_bsearch(key->partnatts,
-													 key->partsupfunc,
-													 key->partcollation,
-													 boundinfo, lower,
-													 &equal);
-
-					if (boundinfo->indexes[offset + 1] < 0)
-					{
-						/*
-						 * Check that the new partition will fit in the gap.
-						 * For it to fit, the new upper bound must be less
-						 * than or equal to the lower bound of the next
-						 * partition, if there is one.
-						 */
-						if (offset + 1 < boundinfo->ndatums)
-						{
-							int32		cmpval;
-							Datum 	   *datums;
-							PartitionRangeDatumKind *kind;
-							bool		is_lower;
-
-							datums = boundinfo->datums[offset + 1];
-							kind = boundinfo->kind[offset + 1];
-							is_lower = (boundinfo->indexes[offset + 1] == -1);
-
-							cmpval = partition_rbound_cmp(key->partnatts,
-														  key->partsupfunc,
-														  key->partcollation,
-														  datums, kind,
-														  is_lower, upper);
-							if (cmpval < 0)
-							{
-								/*
-								 * The new partition overlaps with the
-								 * existing partition between offset + 1 and
-								 * offset + 2.
-								 */
-								overlap = true;
-								with = boundinfo->indexes[offset + 2];
-							}
-						}
-					}
-					else
-					{
-						/*
-						 * The new partition overlaps with the existing
-						 * partition between offset and offset + 1.
-						 */
-						overlap = true;
-						with = boundinfo->indexes[offset + 1];
-					}
-				}
-
-				break;
-			}
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	if (overlap)
-	{
-		Assert(with >= 0);
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" would overlap partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[with])),
-				 parser_errposition(pstate, spec->location)));
-	}
-}
 
 /*
  * check_default_allows_bound
@@ -1505,54 +346,6 @@ map_partition_varattnos(List *expr, int fromrel_varno,
 	return expr;
 }
 
-/*
- * RelationGetPartitionQual
- *
- * Returns a list of partition quals
- */
-List *
-RelationGetPartitionQual(Relation rel)
-{
-	/* Quick exit */
-	if (!rel->rd_rel->relispartition)
-		return NIL;
-
-	return generate_partition_qual(rel);
-}
-
-/*
- * get_partition_qual_relid
- *
- * Returns an expression tree describing the passed-in relation's partition
- * constraint. If there is no partition constraint returns NULL; this can
- * happen if the default partition is the only partition.
- */
-Expr *
-get_partition_qual_relid(Oid relid)
-{
-	Relation	rel = heap_open(relid, AccessShareLock);
-	Expr	   *result = NULL;
-	List	   *and_args;
-
-	/* Do the work only if this relation is a partition. */
-	if (rel->rd_rel->relispartition)
-	{
-		and_args = generate_partition_qual(rel);
-
-		if (and_args == NIL)
-			result = NULL;
-		else if (list_length(and_args) > 1)
-			result = makeBoolExpr(AND_EXPR, and_args, -1);
-		else
-			result = linitial(and_args);
-	}
-
-	/* Keep the lock. */
-	heap_close(rel, NoLock);
-
-	return result;
-}
-
 /* Module-local functions */
 
 /*
@@ -2466,661 +1259,6 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 }
 
 /*
- * generate_partition_qual
- *
- * Generate partition predicate from rel's partition bound expression. The
- * function returns a NIL list if there is no predicate.
- *
- * Result expression tree is stored CacheMemoryContext to ensure it survives
- * as long as the relcache entry. But we should be running in a less long-lived
- * working context. To avoid leaking cache memory if this routine fails partway
- * through, we build in working memory and then copy the completed structure
- * into cache memory.
- */
-static List *
-generate_partition_qual(Relation rel)
-{
-	HeapTuple	tuple;
-	MemoryContext oldcxt;
-	Datum		boundDatum;
-	bool		isnull;
-	PartitionBoundSpec *bound;
-	List	   *my_qual = NIL,
-			   *result = NIL;
-	Relation	parent;
-	bool		found_whole_row;
-
-	/* Guard against stack overflow due to overly deep partition tree */
-	check_stack_depth();
-
-	/* Quick copy */
-	if (rel->rd_partcheck != NIL)
-		return copyObject(rel->rd_partcheck);
-
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
-					   AccessShareLock);
-
-	/* Get pg_class.relpartbound */
-	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(rel));
-
-	boundDatum = SysCacheGetAttr(RELOID, tuple,
-								 Anum_pg_class_relpartbound,
-								 &isnull);
-	if (isnull)					/* should not happen */
-		elog(ERROR, "relation \"%s\" has relpartbound = null",
-			 RelationGetRelationName(rel));
-	bound = castNode(PartitionBoundSpec,
-					 stringToNode(TextDatumGetCString(boundDatum)));
-	ReleaseSysCache(tuple);
-
-	my_qual = get_qual_from_partbound(rel, parent, bound);
-
-	/* 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;
-
-	/*
-	 * Change Vars to have partition's attnos instead of the parent's. We do
-	 * this after we concatenate the parent's quals, because we want every Var
-	 * in it to bear this relation's attnos. It's safe to assume varno = 1
-	 * here.
-	 */
-	result = map_partition_varattnos(result, 1, rel, parent,
-									 &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");
-
-	/* Save a copy in the relcache */
-	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
-	rel->rd_partcheck = copyObject(result);
-	MemoryContextSwitchTo(oldcxt);
-
-	/* Keep the parent locked until commit */
-	heap_close(parent, NoLock);
-
-	return result;
-}
-
-/*
- * get_partition_for_tuple
- *		Finds partition of relation which accepts the partition key specified
- *		in values and isnull
- *
- * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
- * found or -1 if none found.
- */
-int
-get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
-{
-	int			bound_offset;
-	int			part_index = -1;
-	PartitionKey key = RelationGetPartitionKey(relation);
-	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
-
-	/* Route as appropriate based on partitioning strategy. */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				PartitionBoundInfo boundinfo = partdesc->boundinfo;
-				int			greatest_modulus = get_greatest_modulus(boundinfo);
-				uint64		rowHash = compute_hash_value(key->partnatts,
-														 key->partsupfunc,
-														 values, isnull);
-
-				part_index = boundinfo->indexes[rowHash % greatest_modulus];
-			}
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			if (isnull[0])
-			{
-				if (partition_bound_accepts_nulls(partdesc->boundinfo))
-					part_index = partdesc->boundinfo->null_index;
-			}
-			else
-			{
-				bool		equal = false;
-
-				bound_offset = partition_list_bsearch(key->partsupfunc,
-													  key->partcollation,
-													  partdesc->boundinfo,
-													  values[0], &equal);
-				if (bound_offset >= 0 && equal)
-					part_index = partdesc->boundinfo->indexes[bound_offset];
-			}
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				bool		equal = false,
-							range_partkey_has_null = false;
-				int			i;
-
-				/*
-				 * No range includes NULL, so this will be accepted by the
-				 * default partition if there is one, and otherwise rejected.
-				 */
-				for (i = 0; i < key->partnatts; i++)
-				{
-					if (isnull[i])
-					{
-						range_partkey_has_null = true;
-						break;
-					}
-				}
-
-				if (!range_partkey_has_null)
-				{
-					bound_offset = partition_range_datum_bsearch(key->partsupfunc,
-																 key->partcollation,
-																 partdesc->boundinfo,
-																 key->partnatts,
-																 values,
-																 &equal);
-
-					/*
-					 * The bound at bound_offset is less than or equal to the
-					 * tuple value, so the bound at offset+1 is the upper
-					 * bound of the partition we're looking for, if there
-					 * actually exists one.
-					 */
-					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
-				}
-			}
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/*
-	 * part_index < 0 means we failed to find a partition of this parent. Use
-	 * the default partition, if there is one.
-	 */
-	if (part_index < 0)
-		part_index = partdesc->boundinfo->default_index;
-
-	return part_index;
-}
-
-/*
- * Checks if any of the 'attnums' is a partition key attribute for rel
- *
- * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
- * partition key expression.  It's possible for a column to be both used
- * directly and as part of an expression; if that happens, *used_in_expr may
- * end up as either true or false.  That's OK for current uses of this
- * function, because *used_in_expr is only used to tailor the error message
- * text.
- */
-bool
-has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr)
-{
-	PartitionKey key;
-	int			partnatts;
-	List	   *partexprs;
-	ListCell   *partexprs_item;
-	int			i;
-
-	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return false;
-
-	key = RelationGetPartitionKey(rel);
-	partnatts = get_partition_natts(key);
-	partexprs = get_partition_exprs(key);
-
-	partexprs_item = list_head(partexprs);
-	for (i = 0; i < partnatts; i++)
-	{
-		AttrNumber	partattno = get_partition_col_attnum(key, i);
-
-		if (partattno != 0)
-		{
-			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
-							  attnums))
-			{
-				if (used_in_expr)
-					*used_in_expr = false;
-				return true;
-			}
-		}
-		else
-		{
-			/* Arbitrary expression */
-			Node	   *expr = (Node *) lfirst(partexprs_item);
-			Bitmapset  *expr_attrs = NULL;
-
-			/* Find all attributes referenced */
-			pull_varattnos(expr, 1, &expr_attrs);
-			partexprs_item = lnext(partexprs_item);
-
-			if (bms_overlap(attnums, expr_attrs))
-			{
-				if (used_in_expr)
-					*used_in_expr = true;
-				return true;
-			}
-		}
-	}
-
-	return false;
-}
-
-/*
- * qsort_partition_hbound_cmp
- *
- * We sort hash bounds by modulus, then by remainder.
- */
-static int32
-qsort_partition_hbound_cmp(const void *a, const void *b)
-{
-	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
-	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
-
-	return partition_hbound_cmp(h1->modulus, h1->remainder,
-								h2->modulus, h2->remainder);
-}
-
-/*
- * partition_hbound_cmp
- *
- * Compares modulus first, then remainder if modulus are equal.
- */
-static int32
-partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
-{
-	if (modulus1 < modulus2)
-		return -1;
-	if (modulus1 > modulus2)
-		return 1;
-	if (modulus1 == modulus2 && remainder1 != remainder2)
-		return (remainder1 > remainder2) ? 1 : -1;
-	return 0;
-}
-
-/*
- * qsort_partition_list_value_cmp
- *
- * Compare two list partition bound datums
- */
-static int32
-qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
-{
-	Datum		val1 = (*(const PartitionListValue **) a)->value,
-				val2 = (*(const PartitionListValue **) b)->value;
-	PartitionKey key = (PartitionKey) arg;
-
-	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-										   key->partcollation[0],
-										   val1, val2));
-}
-
-/*
- * make_one_range_bound
- *
- * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
- * and a flag telling whether the bound is lower or not.  Made into a function
- * because there are multiple sites that want to use this facility.
- */
-static PartitionRangeBound *
-make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
-{
-	PartitionRangeBound *bound;
-	ListCell   *lc;
-	int			i;
-
-	Assert(datums != NIL);
-
-	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
-	bound->index = index;
-	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
-	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
-													  sizeof(PartitionRangeDatumKind));
-	bound->lower = lower;
-
-	i = 0;
-	foreach(lc, datums)
-	{
-		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
-
-		/* What's contained in this range datum? */
-		bound->kind[i] = datum->kind;
-
-		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
-		{
-			Const	   *val = castNode(Const, datum->value);
-
-			if (val->constisnull)
-				elog(ERROR, "invalid range bound datum");
-			bound->datums[i] = val->constvalue;
-		}
-
-		i++;
-	}
-
-	return bound;
-}
-
-/* Used when sorting range bounds across all range partitions */
-static int32
-qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
-{
-	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
-	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
-	PartitionKey key = (PartitionKey) arg;
-
-	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
-								key->partcollation, b1->datums, b1->kind,
-								b1->lower, b2);
-}
-
-/*
- * partition_rbound_cmp
- *
- * Return for two range bounds whether the 1st one (specified in datums1,
- * kind1, and lower1) is <, =, or > the bound specified in *b2.
- *
- * partnatts, partsupfunc and partcollation give the number of attributes in the
- * bounds to be compared, comparison function to be used and the collations of
- * attributes, respectively.
- *
- * Note that if the values of the two range bounds compare equal, then we take
- * into account whether they are upper or lower bounds, and an upper bound is
- * considered to be smaller than a lower bound. This is important to the way
- * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
- * structure, which only stores the upper bound of a common boundary between
- * two contiguous partitions.
- */
-static int32
-partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2)
-{
-	int32		cmpval = 0;		/* placate compiler */
-	int			i;
-	Datum	   *datums2 = b2->datums;
-	PartitionRangeDatumKind *kind2 = b2->kind;
-	bool		lower2 = b2->lower;
-
-	for (i = 0; i < partnatts; i++)
-	{
-		/*
-		 * First, handle cases where the column is unbounded, which should not
-		 * invoke the comparison procedure, and should not consider any later
-		 * columns. Note that the PartitionRangeDatumKind enum elements
-		 * compare the same way as the values they represent.
-		 */
-		if (kind1[i] < kind2[i])
-			return -1;
-		else if (kind1[i] > kind2[i])
-			return 1;
-		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
-
-			/*
-			 * The column bounds are both MINVALUE or both MAXVALUE. No later
-			 * columns should be considered, but we still need to compare
-			 * whether they are upper or lower bounds.
-			 */
-			break;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
-												 partcollation[i],
-												 datums1[i],
-												 datums2[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	/*
-	 * If the comparison is anything other than equal, we're done. If they
-	 * compare equal though, we still have to consider whether the boundaries
-	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
-	 * two.
-	 */
-	if (cmpval == 0 && lower1 != lower2)
-		cmpval = lower1 ? 1 : -1;
-
-	return cmpval;
-}
-
-/*
- * partition_rbound_datum_cmp
- *
- * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
- * is <, =, or > partition key of tuple (tuple_datums)
- *
- * n_tuple_datums, partsupfunc and partcollation give number of attributes in
- * the bounds to be compared, comparison function to be used and the collations
- * of attributes resp.
- *
- */
-static int32
-partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums)
-{
-	int			i;
-	int32		cmpval = -1;
-
-	for (i = 0; i < n_tuple_datums; i++)
-	{
-		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
-			return -1;
-		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
-			return 1;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
-												 partcollation[i],
-												 rb_datums[i],
-												 tuple_datums[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	return cmpval;
-}
-
-/*
- * partition_list_bsearch
- *		Returns the index of the greatest bound datum that is less than equal
- * 		to the given value or -1 if all of the bound datums are greater
- *
- * *is_equal is set to true if the bound datum at the returned index is equal
- * to the input value.
- */
-static int
-partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
-												 partcollation[0],
-												 boundinfo->datums[mid][0],
-												 value));
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given range bound or -1 if all of the range bounds are
- *		greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input range bound
- */
-static int
-partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
-						Oid *partcollation,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_cmp(partnatts, partsupfunc, partcollation,
-									  boundinfo->datums[mid],
-									  boundinfo->kind[mid],
-									  (boundinfo->indexes[mid] == -1),
-									  probe);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given tuple or -1 if all of the range bounds are greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input tuple.
- */
-static int
-partition_range_datum_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_datum_cmp(partsupfunc,
-											partcollation,
-											boundinfo->datums[mid],
-											boundinfo->kind[mid],
-											values,
-											nvalues);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_hash_bsearch
- *		Returns the index of the greatest (modulus, remainder) pair that is
- *		less than or equal to the given (modulus, remainder) pair or -1 if
- *		all of them are greater
- */
-static int
-partition_hash_bsearch(PartitionBoundInfo boundinfo,
-					   int modulus, int remainder)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval,
-					bound_modulus,
-					bound_remainder;
-
-		mid = (lo + hi + 1) / 2;
-		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
-		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
-		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
-									  modulus, remainder);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-
-			if (cmpval == 0)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * get_default_oid_from_partdesc
- *
- * Given a partition descriptor, return the OID of the default partition, if
- * one exists; else, return InvalidOid.
- */
-Oid
-get_default_oid_from_partdesc(PartitionDesc partdesc)
-{
-	if (partdesc && partdesc->boundinfo &&
-		partition_bound_has_default(partdesc->boundinfo))
-		return partdesc->oids[partdesc->boundinfo->default_index];
-
-	return InvalidOid;
-}
-
-/*
  * get_default_partition_oid
  *
  * Given a relation OID, return the OID of the default partition, if one
@@ -3208,99 +1346,6 @@ get_proposed_default_constraint(List *new_part_constraints)
 }
 
 /*
- * get_partition_bound_num_indexes
- *
- * Returns the number of the entries in the partition bound indexes array.
- */
-static int
-get_partition_bound_num_indexes(PartitionBoundInfo bound)
-{
-	int			num_indexes;
-
-	Assert(bound);
-
-	switch (bound->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-
-			/*
-			 * The number of the entries in the indexes array is same as the
-			 * greatest modulus.
-			 */
-			num_indexes = get_greatest_modulus(bound);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			num_indexes = bound->ndatums;
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			/* Range partitioned table has an extra index. */
-			num_indexes = bound->ndatums + 1;
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) bound->strategy);
-	}
-
-	return num_indexes;
-}
-
-/*
- * get_greatest_modulus
- *
- * Returns the greatest modulus of the hash partition bound. The greatest
- * modulus will be at the end of the datums array because hash partitions are
- * arranged in the ascending order of their modulus and remainders.
- */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
-{
-	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
-	Assert(bound->datums && bound->ndatums > 0);
-	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
-
-	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
-}
-
-/*
- * compute_hash_value
- *
- * Compute the hash value for given not null partition key values.
- */
-static uint64
-compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
-				   Datum *values, bool *isnull)
-{
-	int			i;
-	uint64		rowHash = 0;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-
-	for (i = 0; i < partnatts; i++)
-	{
-		if (!isnull[i])
-		{
-			Datum		hash;
-
-			Assert(OidIsValid(partsupfunc[i].fn_oid));
-
-			/*
-			 * Compute hash for each datum value by calling respective
-			 * datatype-specific hash functions of each partition key
-			 * attribute.
-			 */
-			hash = FunctionCall2(&partsupfunc[i], values[i], seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	return rowHash;
-}
-
-/*
  * satisfies_hash_partition
  *
  * This is an SQL-callable function for use in hash partition constraints.
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 91ba939bdc..6287239f1f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -42,7 +42,6 @@
 #include "access/transam.h"
 #include "access/xact.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_publication.h"
 #include "commands/matview.h"
 #include "commands/trigger.h"
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index ce9a4e16cf..0bb998232c 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -20,6 +20,7 @@
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..f86b6df2ca 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -15,13 +15,13 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/prep.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/memutils.h"
 
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f087369f75..b11d918445 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,7 +33,6 @@
 #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"
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index bd3a0c4a0a..bb2f2619bb 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,7 +27,6 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_statistic_ext.h"
 #include "foreign/fdwapi.h"
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index da8f0f93fc..b471b39667 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -17,7 +17,6 @@
 #include <limits.h>
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
@@ -28,6 +27,7 @@
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
+#include "utils/partcache.h"
 
 
 typedef struct JoinHashEntry
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index b0559ca5bc..1c9a2c469c 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -24,7 +24,6 @@
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
-#include "catalog/partition.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_authid.h"
diff --git a/src/backend/utils/cache/Makefile b/src/backend/utils/cache/Makefile
index a943f8ea4b..94511eaf54 100644
--- a/src/backend/utils/cache/Makefile
+++ b/src/backend/utils/cache/Makefile
@@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
-	relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
-	typcache.o ts_cache.o
+OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o partcache.o \
+    relcache.o relmapper.o relfilenodemap.o spccache.o syscache.o \
+    lsyscache.o typcache.o ts_cache.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
new file mode 100644
index 0000000000..cc284d0ffc
--- /dev/null
+++ b/src/backend/utils/cache/partcache.c
@@ -0,0 +1,2145 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.c
+ *		Partitioning related cache data structures and manipulation functions
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *		  src/backend/utils/cache/partcache.c
+ *
+ *-------------------------------------------------------------------------
+*/
+
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "access/nbtree.h"
+#include "access/sysattr.h"
+#include "catalog/partition.h"
+#include "catalog/pg_inherits_fn.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
+#include "utils/builtins.h"
+#include "utils/datum.h"
+#include "utils/hashutils.h"
+#include "utils/lsyscache.h"
+#include "utils/memutils.h"
+#include "utils/ruleutils.h"
+#include "utils/syscache.h"
+
+/*
+ * Partition bounds are represented using one of the following structs when
+ * they are first read into the backend memory from the catalog (wherein they
+ * are stored in in their parser represenation).  Especially, the bound
+ * comparison and sort functions expect the bounds passed to them to be of
+ * this form.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int			modulus;
+	int			remainder;
+	int			index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+	int			index;
+	Datum		value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+	int			index;
+	Datum	   *datums;			/* range bound datums */
+	PartitionRangeDatumKind *kind;	/* the kind of each datum */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+static List *generate_partition_qual(Relation rel);
+
+static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
+					 int remainder2);
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
+static int partition_hash_bsearch(PartitionBoundInfo boundinfo,
+					   int modulus, int remainder);
+
+static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
+							void *arg);
+static int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal);
+
+static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
+					 List *datums, bool lower);
+static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					 Oid *partcollation, Datum *datums1,
+					 PartitionRangeDatumKind *kind1, bool lower1,
+					 PartitionRangeBound *b2);
+static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
+					void *arg);
+static int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
+						   Oid *partcollation,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums);
+static int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
+						Oid *partcollation,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal);
+static int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
+							  Oid *partcollation,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal);
+
+static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
+
+/*
+ * RelationBuildPartitionKey
+ *		Build and attach to relcache partition key data of relation
+ *
+ * Partitioning key data is a complex structure; to avoid complicated logic to
+ * free individual elements whenever the relcache entry is flushed, we give it
+ * its own memory context, child of CacheMemoryContext, which can easily be
+ * deleted on its own.  To avoid leaking memory in that context in case of an
+ * error partway through this function, the context is initially created as a
+ * child of CurTransactionContext and only re-parented to CacheMemoryContext
+ * at the end, when no further errors are possible.  Also, we don't make this
+ * context the current context except in very brief code sections, out of fear
+ * that some of our callees allocate memory on their own which would be leaked
+ * permanently.
+ */
+void
+RelationBuildPartitionKey(Relation relation)
+{
+	Form_pg_partitioned_table form;
+	HeapTuple	tuple;
+	bool		isnull;
+	int			i;
+	PartitionKey key;
+	AttrNumber *attrs;
+	oidvector  *opclass;
+	oidvector  *collation;
+	ListCell   *partexprs_item;
+	Datum		datum;
+	MemoryContext partkeycxt,
+				oldcxt;
+	int16		procnum;
+
+	tuple = SearchSysCache1(PARTRELID,
+							ObjectIdGetDatum(RelationGetRelid(relation)));
+
+	/*
+	 * The following happens when we have created our pg_class entry but not
+	 * the pg_partitioned_table entry yet.
+	 */
+	if (!HeapTupleIsValid(tuple))
+		return;
+
+	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
+											   RelationGetRelationName(relation),
+											   MEMCONTEXT_COPY_NAME,
+											   ALLOCSET_SMALL_SIZES);
+
+	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
+												sizeof(PartitionKeyData));
+
+	/* Fixed-length attributes */
+	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	key->strategy = form->partstrat;
+	key->partnatts = form->partnatts;
+
+	/*
+	 * We can rely on the first variable-length attribute being mapped to the
+	 * relevant field of the catalog's C struct, because all previous
+	 * attributes are non-nullable and fixed-length.
+	 */
+	attrs = form->partattrs.values;
+
+	/* But use the hard way to retrieve further variable-length attributes */
+	/* Operator class */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partclass, &isnull);
+	Assert(!isnull);
+	opclass = (oidvector *) DatumGetPointer(datum);
+
+	/* Collation */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partcollation, &isnull);
+	Assert(!isnull);
+	collation = (oidvector *) DatumGetPointer(datum);
+
+	/* Expressions */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partexprs, &isnull);
+	if (!isnull)
+	{
+		char	   *exprString;
+		Node	   *expr;
+
+		exprString = TextDatumGetCString(datum);
+		expr = stringToNode(exprString);
+		pfree(exprString);
+
+		/*
+		 * Run the expressions through const-simplification since the planner
+		 * will be comparing them to similarly-processed qual clause operands,
+		 * and may fail to detect valid matches without this step; fix
+		 * opfuncids while at it.  We don't need to bother with
+		 * canonicalize_qual() though, because partition expressions are not
+		 * full-fledged qualification clauses.
+		 */
+		expr = eval_const_expressions(NULL, expr);
+		fix_opfuncids(expr);
+
+		oldcxt = MemoryContextSwitchTo(partkeycxt);
+		key->partexprs = (List *) copyObject(expr);
+		MemoryContextSwitchTo(oldcxt);
+	}
+
+	oldcxt = MemoryContextSwitchTo(partkeycxt);
+	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
+	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
+
+	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+
+	/* Gather type and collation info as well */
+	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
+	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
+	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
+	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
+	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	MemoryContextSwitchTo(oldcxt);
+
+	/* determine support function number to search for */
+	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
+		HASHEXTENDED_PROC : BTORDER_PROC;
+
+	/* Copy partattrs and fill other per-attribute info */
+	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		AttrNumber	attno = key->partattrs[i];
+		HeapTuple	opclasstup;
+		Form_pg_opclass opclassform;
+		Oid			funcid;
+
+		/* Collect opfamily information */
+		opclasstup = SearchSysCache1(CLAOID,
+									 ObjectIdGetDatum(opclass->values[i]));
+		if (!HeapTupleIsValid(opclasstup))
+			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
+
+		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
+		key->partopfamily[i] = opclassform->opcfamily;
+		key->partopcintype[i] = opclassform->opcintype;
+
+		/* Get a support function for the specified opfamily and datatypes */
+		funcid = get_opfamily_proc(opclassform->opcfamily,
+								   opclassform->opcintype,
+								   opclassform->opcintype,
+								   procnum);
+		if (!OidIsValid(funcid))
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
+							NameStr(opclassform->opcname),
+							(key->strategy == PARTITION_STRATEGY_HASH) ?
+							"hash" : "btree",
+							procnum,
+							format_type_be(opclassform->opcintype))));
+
+		fmgr_info(funcid, &key->partsupfunc[i]);
+
+		/* Collation */
+		key->partcollation[i] = collation->values[i];
+
+		/* Collect type information */
+		if (attno != 0)
+		{
+			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
+
+			key->parttypid[i] = att->atttypid;
+			key->parttypmod[i] = att->atttypmod;
+			key->parttypcoll[i] = att->attcollation;
+		}
+		else
+		{
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			key->parttypid[i] = exprType(lfirst(partexprs_item));
+			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
+			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
+
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(key->parttypid[i],
+							 &key->parttyplen[i],
+							 &key->parttypbyval[i],
+							 &key->parttypalign[i]);
+
+		ReleaseSysCache(opclasstup);
+	}
+
+	ReleaseSysCache(tuple);
+
+	/*
+	 * Success --- reparent our context and make the relcache point to the
+	 * newly constructed key
+	 */
+	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
+	relation->rd_partkeycxt = partkeycxt;
+	relation->rd_partkey = key;
+}
+
+/*
+ * RelationBuildPartitionDesc
+ *		Form rel's partition descriptor
+ *
+ * Not flushed from the cache by RelationClearRelation() unless changed because
+ * of addition or removal of partition.
+ */
+void
+RelationBuildPartitionDesc(Relation rel)
+{
+	List	   *inhoids,
+			   *partoids;
+	Oid		   *oids = NULL;
+	List	   *boundspecs = NIL;
+	ListCell   *cell;
+	int			i,
+				nparts;
+	PartitionKey key = RelationGetPartitionKey(rel);
+	PartitionDesc result;
+	MemoryContext oldcxt;
+
+	int			ndatums = 0;
+	int			default_index = -1;
+
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
+	/* List partitioning specific */
+	PartitionListValue **all_values = NULL;
+	int			null_index = -1;
+
+	/* Range partitioning specific */
+	PartitionRangeBound **rbounds = NULL;
+
+	/*
+	 * The following could happen in situations where rel has a pg_class entry
+	 * but not the pg_partitioned_table entry yet.
+	 */
+	if (key == NULL)
+		return;
+
+	/* Get partition oids from pg_inherits */
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+
+	/* Collect bound spec nodes in a list */
+	i = 0;
+	partoids = NIL;
+	foreach(cell, inhoids)
+	{
+		Oid			inhrelid = lfirst_oid(cell);
+		HeapTuple	tuple;
+		Datum		datum;
+		bool		isnull;
+		Node	   *boundspec;
+
+		tuple = SearchSysCache1(RELOID, inhrelid);
+		if (!HeapTupleIsValid(tuple))
+			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+		/*
+		 * It is possible that the pg_class tuple of a partition has not been
+		 * updated yet to set its relpartbound field.  The only case where
+		 * this happens is when we open the parent relation to check using its
+		 * partition descriptor that a new partition's bound does not overlap
+		 * some existing partition.
+		 */
+		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		{
+			ReleaseSysCache(tuple);
+			continue;
+		}
+
+		datum = SysCacheGetAttr(RELOID, tuple,
+								Anum_pg_class_relpartbound,
+								&isnull);
+		Assert(!isnull);
+		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+		/*
+		 * Sanity check: If the PartitionBoundSpec says this is the default
+		 * partition, its OID should correspond to whatever's stored in
+		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 */
+		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		{
+			Oid			partdefid;
+
+			partdefid = get_default_partition_oid(RelationGetRelid(rel));
+			if (partdefid != inhrelid)
+				elog(ERROR, "expected partdefid %u, but got %u",
+					 inhrelid, partdefid);
+		}
+
+		boundspecs = lappend(boundspecs, boundspec);
+		partoids = lappend_oid(partoids, inhrelid);
+		ReleaseSysCache(tuple);
+	}
+
+	nparts = list_length(partoids);
+
+	if (nparts > 0)
+	{
+		oids = (Oid *) palloc(nparts * sizeof(Oid));
+		i = 0;
+		foreach(cell, partoids)
+			oids[i++] = lfirst_oid(cell);
+
+		/* Convert from node to the internal representation */
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **)
+				palloc(nparts * sizeof(PartitionHashBound *));
+
+			i = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
+		{
+			List	   *non_null_values = NIL;
+
+			/*
+			 * Create a unified list of non-null values across all partitions.
+			 */
+			i = 0;
+			null_index = -1;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				ListCell   *c;
+
+				if (spec->strategy != PARTITION_STRATEGY_LIST)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
+				foreach(c, spec->listdatums)
+				{
+					Const	   *val = castNode(Const, lfirst(c));
+					PartitionListValue *list_value = NULL;
+
+					if (!val->constisnull)
+					{
+						list_value = (PartitionListValue *)
+							palloc0(sizeof(PartitionListValue));
+						list_value->index = i;
+						list_value->value = val->constvalue;
+					}
+					else
+					{
+						/*
+						 * Never put a null into the values array, flag
+						 * instead for the code further down below where we
+						 * construct the actual relcache struct.
+						 */
+						if (null_index != -1)
+							elog(ERROR, "found null more than once");
+						null_index = i;
+					}
+
+					if (list_value)
+						non_null_values = lappend(non_null_values,
+												  list_value);
+				}
+
+				i++;
+			}
+
+			ndatums = list_length(non_null_values);
+
+			/*
+			 * Collect all list values in one array. Alongside the value, we
+			 * also save the index of partition the value comes from.
+			 */
+			all_values = (PartitionListValue **) palloc(ndatums *
+														sizeof(PartitionListValue *));
+			i = 0;
+			foreach(cell, non_null_values)
+			{
+				PartitionListValue *src = lfirst(cell);
+
+				all_values[i] = (PartitionListValue *)
+					palloc(sizeof(PartitionListValue));
+				all_values[i]->value = src->value;
+				all_values[i]->index = src->index;
+				i++;
+			}
+
+			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
+					  qsort_partition_list_value_cmp, (void *) key);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_RANGE)
+		{
+			int			k;
+			PartitionRangeBound **all_bounds,
+					   *prev;
+
+			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
+														  sizeof(PartitionRangeBound *));
+
+			/*
+			 * Create a unified list of range bounds across all the
+			 * partitions.
+			 */
+			i = ndatums = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				PartitionRangeBound *lower,
+						   *upper;
+
+				if (spec->strategy != PARTITION_STRATEGY_RANGE)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the allbounds array
+				 * for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i++;
+					continue;
+				}
+
+				lower = make_one_range_bound(key, i, spec->lowerdatums,
+											 true);
+				upper = make_one_range_bound(key, i, spec->upperdatums,
+											 false);
+				all_bounds[ndatums++] = lower;
+				all_bounds[ndatums++] = upper;
+				i++;
+			}
+
+			Assert(ndatums == nparts * 2 ||
+				   (default_index != -1 && ndatums == (nparts - 1) * 2));
+
+			/* Sort all the bounds in ascending order */
+			qsort_arg(all_bounds, ndatums,
+					  sizeof(PartitionRangeBound *),
+					  qsort_partition_rbound_cmp,
+					  (void *) key);
+
+			/* Save distinct bounds from all_bounds into rbounds. */
+			rbounds = (PartitionRangeBound **)
+				palloc(ndatums * sizeof(PartitionRangeBound *));
+			k = 0;
+			prev = NULL;
+			for (i = 0; i < ndatums; i++)
+			{
+				PartitionRangeBound *cur = all_bounds[i];
+				bool		is_distinct = false;
+				int			j;
+
+				/* Is the current bound distinct from the previous one? */
+				for (j = 0; j < key->partnatts; j++)
+				{
+					Datum		cmpval;
+
+					if (prev == NULL || cur->kind[j] != prev->kind[j])
+					{
+						is_distinct = true;
+						break;
+					}
+
+					/*
+					 * If the bounds are both MINVALUE or MAXVALUE, stop now
+					 * and treat them as equal, since any values after this
+					 * point must be ignored.
+					 */
+					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
+						break;
+
+					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
+											   key->partcollation[j],
+											   cur->datums[j],
+											   prev->datums[j]);
+					if (DatumGetInt32(cmpval) != 0)
+					{
+						is_distinct = true;
+						break;
+					}
+				}
+
+				/*
+				 * Only if the bound is distinct save it into a temporary
+				 * array i.e. rbounds which is later copied into boundinfo
+				 * datums array.
+				 */
+				if (is_distinct)
+					rbounds[k++] = all_bounds[i];
+
+				prev = cur;
+			}
+
+			/* Update ndatums to hold the count of distinct datums. */
+			ndatums = k;
+		}
+		else
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/* Now build the actual relcache partition descriptor */
+	rel->rd_pdcxt = AllocSetContextCreateExtended(CacheMemoryContext,
+												  RelationGetRelationName(rel),
+												  MEMCONTEXT_COPY_NAME,
+												  ALLOCSET_DEFAULT_SIZES);
+	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+
+	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
+	result->nparts = nparts;
+	if (nparts > 0)
+	{
+		PartitionBoundInfo boundinfo;
+		int		   *mapping;
+		int			next_index = 0;
+
+		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
+
+		boundinfo = (PartitionBoundInfoData *)
+			palloc0(sizeof(PartitionBoundInfoData));
+		boundinfo->strategy = key->strategy;
+		boundinfo->default_index = -1;
+		boundinfo->ndatums = ndatums;
+		boundinfo->null_index = -1;
+		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
+
+		/* Initialize mapping array with invalid values */
+		mapping = (int *) palloc(sizeof(int) * nparts);
+		for (i = 0; i < nparts; i++)
+			mapping[i] = -1;
+
+		switch (key->strategy)
+		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+
+					for (i = 0; i < greatest_modulus; i++)
+						boundinfo->indexes[i] = -1;
+
+					for (i = 0; i < nparts; i++)
+					{
+						int			modulus = hbounds[i]->modulus;
+						int			remainder = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(modulus);
+						boundinfo->datums[i][1] = Int32GetDatum(remainder);
+
+						while (remainder < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[remainder] == -1);
+							boundinfo->indexes[remainder] = i;
+							remainder += modulus;
+						}
+
+						mapping[hbounds[i]->index] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
+			case PARTITION_STRATEGY_LIST:
+				{
+					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
+
+					/*
+					 * Copy values.  Indexes of individual values are mapped
+					 * to canonical values so that they match for any two list
+					 * partitioned tables with same number of partitions and
+					 * same lists per partition.  One way to canonicalize is
+					 * to assign the index in all_values[] of the smallest
+					 * value of each partition, as the index of all of the
+					 * partition's values.
+					 */
+					for (i = 0; i < ndatums; i++)
+					{
+						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
+						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
+															key->parttypbyval[0],
+															key->parttyplen[0]);
+
+						/* If the old index has no mapping, assign one */
+						if (mapping[all_values[i]->index] == -1)
+							mapping[all_values[i]->index] = next_index++;
+
+						boundinfo->indexes[i] = mapping[all_values[i]->index];
+					}
+
+					/*
+					 * If null-accepting partition has no mapped index yet,
+					 * assign one.  This could happen if such partition
+					 * accepts only null and hence not covered in the above
+					 * loop which only handled non-null values.
+					 */
+					if (null_index != -1)
+					{
+						Assert(null_index >= 0);
+						if (mapping[null_index] == -1)
+							mapping[null_index] = next_index++;
+						boundinfo->null_index = mapping[null_index];
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any value not
+						 * specified in the lists of other partitions, hence
+						 * it should not get mapped index while assigning
+						 * those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+
+					/* All partition must now have a valid mapping */
+					Assert(next_index == nparts);
+					break;
+				}
+
+			case PARTITION_STRATEGY_RANGE:
+				{
+					boundinfo->kind = (PartitionRangeDatumKind **)
+						palloc(ndatums *
+							   sizeof(PartitionRangeDatumKind *));
+					boundinfo->indexes = (int *) palloc((ndatums + 1) *
+														sizeof(int));
+
+					for (i = 0; i < ndatums; i++)
+					{
+						int			j;
+
+						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
+																sizeof(Datum));
+						boundinfo->kind[i] = (PartitionRangeDatumKind *)
+							palloc(key->partnatts *
+								   sizeof(PartitionRangeDatumKind));
+						for (j = 0; j < key->partnatts; j++)
+						{
+							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
+								boundinfo->datums[i][j] =
+									datumCopy(rbounds[i]->datums[j],
+											  key->parttypbyval[j],
+											  key->parttyplen[j]);
+							boundinfo->kind[i][j] = rbounds[i]->kind[j];
+						}
+
+						/*
+						 * There is no mapping for invalid indexes.
+						 *
+						 * Any lower bounds in the rbounds array have invalid
+						 * indexes assigned, because the values between the
+						 * previous bound (if there is one) and this (lower)
+						 * bound are not part of the range of any existing
+						 * partition.
+						 */
+						if (rbounds[i]->lower)
+							boundinfo->indexes[i] = -1;
+						else
+						{
+							int			orig_index = rbounds[i]->index;
+
+							/* If the old index has no mapping, assign one */
+							if (mapping[orig_index] == -1)
+								mapping[orig_index] = next_index++;
+
+							boundinfo->indexes[i] = mapping[orig_index];
+						}
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						Assert(default_index >= 0 && mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					boundinfo->indexes[i] = -1;
+					break;
+				}
+
+			default:
+				elog(ERROR, "unexpected partition strategy: %d",
+					 (int) key->strategy);
+		}
+
+		result->boundinfo = boundinfo;
+
+		/*
+		 * Now assign OIDs from the original array into mapped indexes of the
+		 * result array.  Order of OIDs in the former is defined by the
+		 * catalog scan that retrieved them, whereas that in the latter is
+		 * defined by canonicalized representation of the partition bounds.
+		 */
+		for (i = 0; i < nparts; i++)
+			result->oids[mapping[i]] = oids[i];
+		pfree(mapping);
+	}
+
+	MemoryContextSwitchTo(oldcxt);
+	rel->rd_partdesc = result;
+}
+
+/*
+ * Are two partition bound collections logically equal?
+ *
+ * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
+ * This is also useful when b1 and b2 are bound collections of two separate
+ * relations, respectively, because PartitionBoundInfo is a canonical
+ * representation of partition bounds.
+ */
+bool
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
+					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			i;
+
+	if (b1->strategy != b2->strategy)
+		return false;
+
+	if (b1->ndatums != b2->ndatums)
+		return false;
+
+	if (b1->null_index != b2->null_index)
+		return false;
+
+	if (b1->default_index != b2->default_index)
+		return false;
+
+	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	{
+		int			greatest_modulus = get_greatest_modulus(b1);
+
+		/*
+		 * If two hash partitioned tables have different greatest moduli,
+		 * their partition schemes don't match.
+		 */
+		if (greatest_modulus != get_greatest_modulus(b2))
+			return false;
+
+		/*
+		 * We arrange the partitions in the ascending order of their modulus
+		 * and remainders.  Also every modulus is factor of next larger
+		 * modulus.  Therefore we can safely store index of a given partition
+		 * in indexes array at remainder of that partition.  Also entries at
+		 * (remainder + N * modulus) positions in indexes array are all same
+		 * for (modulus, remainder) specification for any partition.  Thus
+		 * datums array from both the given bounds are same, if and only if
+		 * their indexes array will be same.  So, it suffices to compare
+		 * indexes array.
+		 */
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Nonetheless make sure that the bounds are indeed same when the
+		 * indexes match.  Hash partition bound stores modulus and remainder
+		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			Assert((b1->datums[i][0] == b2->datums[i][0] &&
+					b1->datums[i][1] == b2->datums[i][1]));
+#endif
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
+		{
+			int			j;
+
+			for (j = 0; j < partnatts; j++)
+			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->kind != NULL)
+				{
+					/* The different kinds of bound all differ from each other */
+					if (b1->kind[i][j] != b2->kind[i][j])
+						return false;
+
+					/*
+					 * Non-finite bounds are equal without further
+					 * examination.
+					 */
+					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
+						continue;
+				}
+
+				/*
+				 * Compare the actual values. Note that it would be both
+				 * incorrect and unsafe to invoke the comparison operator
+				 * derived from the partitioning specification here.  It would
+				 * be incorrect because we want the relcache entry to be
+				 * updated for ANY change to the partition bounds, not just
+				 * those that the partitioning operator thinks are
+				 * significant.  It would be unsafe because we might reach
+				 * this code in the context of an aborted transaction, and an
+				 * arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be
+				 * safe.
+				 */
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  parttypbyval[j], parttyplen[j]))
+					return false;
+			}
+
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+		}
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
+			return false;
+	}
+	return true;
+}
+
+/*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specification.
+ */
+PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo dest;
+	int			i;
+	int			ndatums;
+	int			partnatts;
+	int			num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	num_indexes = get_partition_bound_num_indexes(src);
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+														 sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+															   sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int			j;
+
+		/*
+		 * For a corresponding to hash partition, datums array will have two
+		 * elements - modulus and remainder.
+		 */
+		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
+		int			natts = hash_part ? 2 : partnatts;
+
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+
+		for (j = 0; j < natts; j++)
+		{
+			bool		byval;
+			int			typlen;
+
+			if (hash_part)
+			{
+				typlen = sizeof(int32); /* Always int4 */
+				byval = true;	/* int4 is pass-by-value */
+			}
+			else
+			{
+				byval = key->parttypbyval[j];
+				typlen = key->parttyplen[j];
+			}
+
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   byval, typlen);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
+ * check_new_partition_bound
+ *
+ * Checks if the new partition's bound overlaps any of the existing partitions
+ * of parent.  Also performs additional checks as necessary per strategy.
+ */
+void
+check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
+	ParseState *pstate = make_parsestate(NULL);
+	int			with = -1;
+	bool		overlap = false;
+
+	if (spec->is_default)
+	{
+		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+			return;
+
+		/* Default partition already exists, error out. */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+				 parser_errposition(pstate, spec->location)));
+	}
+
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					Datum	  **datums = boundinfo->datums;
+					int			ndatums = boundinfo->ndatums;
+					int			greatest_modulus;
+					int			remainder;
+					int			offset;
+					bool		valid_modulus = true;
+					int			prev_modulus,	/* Previous largest modulus */
+								next_modulus;	/* Next largest modulus */
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.
+					 *
+					 * Get the greatest (modulus, remainder) pair contained in
+					 * boundinfo->datums that is less than or equal to the
+					 * (spec->modulus, spec->remainder) pair.
+					 */
+					offset = partition_hash_bsearch(boundinfo,
+													spec->modulus,
+													spec->remainder);
+					if (offset < 0)
+					{
+						next_modulus = DatumGetInt32(datums[0][0]);
+						valid_modulus = (next_modulus % spec->modulus) == 0;
+					}
+					else
+					{
+						prev_modulus = DatumGetInt32(datums[offset][0]);
+						valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+						if (valid_modulus && (offset + 1) < ndatums)
+						{
+							next_modulus = DatumGetInt32(datums[offset + 1][0]);
+							valid_modulus = (next_modulus % spec->modulus) == 0;
+						}
+					}
+
+					if (!valid_modulus)
+						ereport(ERROR,
+								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
+
+					greatest_modulus = get_greatest_modulus(boundinfo);
+					remainder = spec->remainder;
+
+					/*
+					 * Normally, the lowest remainder that could conflict with
+					 * the new partition is equal to the remainder specified
+					 * for the new partition, but when the new partition has a
+					 * modulus higher than any used so far, we need to adjust.
+					 */
+					if (remainder >= greatest_modulus)
+						remainder = remainder % greatest_modulus;
+
+					/* Check every potentially-conflicting remainder. */
+					do
+					{
+						if (boundinfo->indexes[remainder] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[remainder];
+							break;
+						}
+						remainder += spec->modulus;
+					} while (remainder < greatest_modulus);
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_LIST:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+
+				if (partdesc->nparts > 0)
+				{
+					ListCell   *cell;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					foreach(cell, spec->listdatums)
+					{
+						Const	   *val = castNode(Const, lfirst(cell));
+
+						if (!val->constisnull)
+						{
+							int			offset;
+							bool		equal;
+
+							offset = partition_list_bsearch(key->partsupfunc,
+														key->partcollation,
+															boundinfo,
+															val->constvalue,
+															&equal);
+							if (offset >= 0 && equal)
+							{
+								overlap = true;
+								with = boundinfo->indexes[offset];
+								break;
+							}
+						}
+						else if (partition_bound_accepts_nulls(boundinfo))
+						{
+							overlap = true;
+							with = boundinfo->null_index;
+							break;
+						}
+					}
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				PartitionRangeBound *lower,
+						   *upper;
+
+				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
+				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
+
+				/*
+				 * First check if the resulting range would be empty with
+				 * specified lower and upper bounds
+				 */
+				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
+										 key->partcollation, lower->datums,
+										 lower->kind, true, upper) >= 0)
+				{
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("empty range bound specified for partition \"%s\"",
+									relname),
+							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
+									   get_range_partbound_string(spec->lowerdatums),
+									   get_range_partbound_string(spec->upperdatums)),
+							 parser_errposition(pstate, spec->location)));
+				}
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					int			offset;
+					bool		equal;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Test whether the new lower bound (which is treated
+					 * inclusively as part of the new partition) lies inside
+					 * an existing partition, or in a gap.
+					 *
+					 * If it's inside an existing partition, the bound at
+					 * offset + 1 will be the upper bound of that partition,
+					 * and its index will be >= 0.
+					 *
+					 * If it's in a gap, the bound at offset + 1 will be the
+					 * lower bound of the next partition, and its index will
+					 * be -1. This is also true if there is no next partition,
+					 * since the index array is initialised with an extra -1
+					 * at the end.
+					 */
+					offset = partition_range_bsearch(key->partnatts,
+													 key->partsupfunc,
+													 key->partcollation,
+													 boundinfo, lower,
+													 &equal);
+
+					if (boundinfo->indexes[offset + 1] < 0)
+					{
+						/*
+						 * Check that the new partition will fit in the gap.
+						 * For it to fit, the new upper bound must be less
+						 * than or equal to the lower bound of the next
+						 * partition, if there is one.
+						 */
+						if (offset + 1 < boundinfo->ndatums)
+						{
+							int32		cmpval;
+							Datum 	   *datums;
+							PartitionRangeDatumKind *kind;
+							bool		is_lower;
+
+							datums = boundinfo->datums[offset + 1];
+							kind = boundinfo->kind[offset + 1];
+							is_lower = (boundinfo->indexes[offset + 1] == -1);
+
+							cmpval = partition_rbound_cmp(key->partnatts,
+														  key->partsupfunc,
+														  key->partcollation,
+														  datums, kind,
+														  is_lower, upper);
+							if (cmpval < 0)
+							{
+								/*
+								 * The new partition overlaps with the
+								 * existing partition between offset + 1 and
+								 * offset + 2.
+								 */
+								overlap = true;
+								with = boundinfo->indexes[offset + 2];
+							}
+						}
+					}
+					else
+					{
+						/*
+						 * The new partition overlaps with the existing
+						 * partition between offset and offset + 1.
+						 */
+						overlap = true;
+						with = boundinfo->indexes[offset + 1];
+					}
+				}
+
+				break;
+			}
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	if (overlap)
+	{
+		Assert(with >= 0);
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" would overlap partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[with])),
+				 parser_errposition(pstate, spec->location)));
+	}
+}
+
+/*
+ * RelationGetPartitionQual
+ *
+ * Returns a list of partition quals
+ */
+List *
+RelationGetPartitionQual(Relation rel)
+{
+	/* Quick exit */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
+	return generate_partition_qual(rel);
+}
+
+/*
+ * get_partition_qual_relid
+ *
+ * Returns an expression tree describing the passed-in relation's partition
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
+ */
+Expr *
+get_partition_qual_relid(Oid relid)
+{
+	Relation	rel = heap_open(relid, AccessShareLock);
+	Expr	   *result = NULL;
+	List	   *and_args;
+
+	/* Do the work only if this relation is a partition. */
+	if (rel->rd_rel->relispartition)
+	{
+		and_args = generate_partition_qual(rel);
+
+		if (and_args == NIL)
+			result = NULL;
+		else if (list_length(and_args) > 1)
+			result = makeBoolExpr(AND_EXPR, and_args, -1);
+		else
+			result = linitial(and_args);
+	}
+
+	/* Keep the lock. */
+	heap_close(rel, NoLock);
+
+	return result;
+}
+
+/*
+ * Checks if any of the 'attnums' is a partition key attribute for rel
+ *
+ * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
+ * partition key expression.  It's possible for a column to be both used
+ * directly and as part of an expression; if that happens, *used_in_expr may
+ * end up as either true or false.  That's OK for current uses of this
+ * function, because *used_in_expr is only used to tailor the error message
+ * text.
+ */
+bool
+has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr)
+{
+	PartitionKey key;
+	int			partnatts;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	int			i;
+
+	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		return false;
+
+	key = RelationGetPartitionKey(rel);
+	partnatts = get_partition_natts(key);
+	partexprs = get_partition_exprs(key);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+		{
+			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
+							  attnums))
+			{
+				if (used_in_expr)
+					*used_in_expr = false;
+				return true;
+			}
+		}
+		else
+		{
+			/* Arbitrary expression */
+			Node	   *expr = (Node *) lfirst(partexprs_item);
+			Bitmapset  *expr_attrs = NULL;
+
+			/* Find all attributes referenced */
+			pull_varattnos(expr, 1, &expr_attrs);
+			partexprs_item = lnext(partexprs_item);
+
+			if (bms_overlap(attnums, expr_attrs))
+			{
+				if (used_in_expr)
+					*used_in_expr = true;
+				return true;
+			}
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple
+ *		Finds partition of relation which accepts the partition key specified
+ *		in values and isnull
+ *
+ * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
+ * found or -1 if none found.
+ */
+int
+get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
+{
+	int			bound_offset;
+	int			part_index = -1;
+	PartitionKey key = RelationGetPartitionKey(relation);
+	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
+
+	/* Route as appropriate based on partitioning strategy. */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundInfo boundinfo = partdesc->boundinfo;
+				int			greatest_modulus = get_greatest_modulus(boundinfo);
+				uint64		rowHash = compute_hash_value(key, values, isnull);
+
+				part_index = boundinfo->indexes[rowHash % greatest_modulus];
+			}
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			if (isnull[0])
+			{
+				if (partition_bound_accepts_nulls(partdesc->boundinfo))
+					part_index = partdesc->boundinfo->null_index;
+			}
+			else
+			{
+				bool		equal = false;
+
+				bound_offset = partition_list_bsearch(key->partsupfunc,
+													  key->partcollation,
+													  partdesc->boundinfo,
+													  values[0], &equal);
+				if (bound_offset >= 0 && equal)
+					part_index = partdesc->boundinfo->indexes[bound_offset];
+			}
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				bool		equal = false,
+							range_partkey_has_null = false;
+				int			i;
+
+				/*
+				 * No range includes NULL, so this will be accepted by the
+				 * default partition if there is one, and otherwise rejected.
+				 */
+				for (i = 0; i < key->partnatts; i++)
+				{
+					if (isnull[i])
+					{
+						range_partkey_has_null = true;
+						break;
+					}
+				}
+
+				if (!range_partkey_has_null)
+				{
+					bound_offset =
+							partition_range_datum_bsearch(key->partsupfunc,
+														  key->partcollation,
+														  partdesc->boundinfo,
+														  key->partnatts,
+														  values,
+														  &equal);
+					/*
+					 * The bound at bound_offset is less than or equal to the
+					 * tuple value, so the bound at offset+1 is the upper
+					 * bound of the partition we're looking for, if there
+					 * actually exists one.
+					 */
+					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
+				}
+			}
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/*
+	 * part_index < 0 means we failed to find a partition of this parent. Use
+	 * the default partition, if there is one.
+	 */
+	if (part_index < 0)
+		part_index = partdesc->boundinfo->default_index;
+
+	return part_index;
+}
+
+/*
+ * get_greatest_modulus
+ *
+ * Returns the greatest modulus of the hash partition bound. The greatest
+ * modulus will be at the end of the datums array because hash partitions are
+ * arranged in the ascending order of their modulus and remainders.
+ */
+int
+get_greatest_modulus(PartitionBoundInfo bound)
+{
+	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
+	Assert(bound->datums && bound->ndatums > 0);
+	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
+
+	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+uint64
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+	int			i;
+	int			nkeys = key->partnatts;
+	uint64		rowHash = 0;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+	for (i = 0; i < nkeys; i++)
+	{
+		if (!isnull[i])
+		{
+			Datum		hash;
+
+			Assert(OidIsValid(key->partsupfunc[i].fn_oid));
+
+			/*
+			 * Compute hash for each datum value by calling respective
+			 * datatype-specific hash functions of each partition key
+			 * attribute.
+			 */
+			hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+	if (partdesc && partdesc->boundinfo &&
+		partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
+
+/* Module-local functions. */
+
+/*
+ * generate_partition_qual
+ *
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
+ *
+ * Result expression tree is stored CacheMemoryContext to ensure it survives
+ * as long as the relcache entry. But we should be running in a less long-lived
+ * working context. To avoid leaking cache memory if this routine fails partway
+ * through, we build in working memory and then copy the completed structure
+ * into cache memory.
+ */
+static List *
+generate_partition_qual(Relation rel)
+{
+	HeapTuple	tuple;
+	MemoryContext oldcxt;
+	Datum		boundDatum;
+	bool		isnull;
+	PartitionBoundSpec *bound;
+	List	   *my_qual = NIL,
+			   *result = NIL;
+	Relation	parent;
+	bool		found_whole_row;
+
+	/* Guard against stack overflow due to overly deep partition tree */
+	check_stack_depth();
+
+	/* Quick copy */
+	if (rel->rd_partcheck != NIL)
+		return copyObject(rel->rd_partcheck);
+
+	/* Grab at least an AccessShareLock on the parent table */
+	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
+					   AccessShareLock);
+
+	/* Get pg_class.relpartbound */
+	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	boundDatum = SysCacheGetAttr(RELOID, tuple,
+								 Anum_pg_class_relpartbound,
+								 &isnull);
+	if (isnull)					/* should not happen */
+		elog(ERROR, "relation \"%s\" has relpartbound = null",
+			 RelationGetRelationName(rel));
+	bound = castNode(PartitionBoundSpec,
+					 stringToNode(TextDatumGetCString(boundDatum)));
+	ReleaseSysCache(tuple);
+
+	my_qual = get_qual_from_partbound(rel, parent, bound);
+
+	/* 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;
+
+	/*
+	 * Change Vars to have partition's attnos instead of the parent's. We do
+	 * this after we concatenate the parent's quals, because we want every Var
+	 * in it to bear this relation's attnos. It's safe to assume varno = 1
+	 * here.
+	 */
+	result = map_partition_varattnos(result, 1, rel, parent,
+									 &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");
+
+	/* Save a copy in the relcache */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	rel->rd_partcheck = copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* Keep the parent locked until commit */
+	heap_close(parent, NoLock);
+
+	return result;
+}
+
+/*
+ * partition_hbound_cmp
+ *
+ * Compares modulus first, then remainder if modulus are equal.
+ */
+static int32
+partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
+{
+	if (modulus1 < modulus2)
+		return -1;
+	if (modulus1 > modulus2)
+		return 1;
+	if (modulus1 == modulus2 && remainder1 != remainder2)
+		return (remainder1 > remainder2) ? 1 : -1;
+	return 0;
+}
+
+/*
+ * qsort_partition_hbound_cmp
+ *
+ * We sort hash bounds by modulus, then by remainder.
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+
+	return partition_hbound_cmp(h1->modulus, h1->remainder,
+								h2->modulus, h2->remainder);
+}
+
+/*
+ * qsort_partition_list_value_cmp
+ *
+ * Compare two list partition bound datums
+ */
+static int32
+qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+{
+	Datum		val1 = (*(const PartitionListValue **) a)->value,
+				val2 = (*(const PartitionListValue **) b)->value;
+	PartitionKey key = (PartitionKey) arg;
+
+	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+										   key->partcollation[0],
+										   val1, val2));
+}
+
+/*
+ * make_one_range_bound
+ *
+ * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
+ * and a flag telling whether the bound is lower or not.  Made into a function
+ * because there are multiple sites that want to use this facility.
+ */
+static PartitionRangeBound *
+make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
+{
+	PartitionRangeBound *bound;
+	ListCell   *lc;
+	int			i;
+
+	Assert(datums != NIL);
+
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
+	bound->index = index;
+	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
+	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
+													  sizeof(PartitionRangeDatumKind));
+	bound->lower = lower;
+
+	i = 0;
+	foreach(lc, datums)
+	{
+		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+
+		/* What's contained in this range datum? */
+		bound->kind[i] = datum->kind;
+
+		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		{
+			Const	   *val = castNode(Const, datum->value);
+
+			if (val->constisnull)
+				elog(ERROR, "invalid range bound datum");
+			bound->datums[i] = val->constvalue;
+		}
+
+		i++;
+	}
+
+	return bound;
+}
+
+/*
+ * partition_rbound_cmp
+ *
+ * Return for two range bounds whether the 1st one (specified in datums1,
+ * kind1, and lower1) is <, =, or > the bound specified in *b2.
+ *
+ * partnatts, partsupfunc and partcollation give the number of attributes in the
+ * bounds to be compared, comparison function to be used and the collations of
+ * attributes, respectively.
+ *
+ * Note that if the values of the two range bounds compare equal, then we take
+ * into account whether they are upper or lower bounds, and an upper bound is
+ * considered to be smaller than a lower bound. This is important to the way
+ * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
+ * structure, which only stores the upper bound of a common boundary between
+ * two contiguous partitions.
+ */
+static int32
+partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2)
+{
+	int32		cmpval = 0;		/* placate compiler */
+	int			i;
+	Datum	   *datums2 = b2->datums;
+	PartitionRangeDatumKind *kind2 = b2->kind;
+	bool		lower2 = b2->lower;
+
+	for (i = 0; i < partnatts; i++)
+	{
+		/*
+		 * First, handle cases where the column is unbounded, which should not
+		 * invoke the comparison procedure, and should not consider any later
+		 * columns. Note that the PartitionRangeDatumKind enum elements
+		 * compare the same way as the values they represent.
+		 */
+		if (kind1[i] < kind2[i])
+			return -1;
+		else if (kind1[i] > kind2[i])
+			return 1;
+		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+
+			/*
+			 * The column bounds are both MINVALUE or both MAXVALUE. No later
+			 * columns should be considered, but we still need to compare
+			 * whether they are upper or lower bounds.
+			 */
+			break;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
+												 datums1[i],
+												 datums2[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	/*
+	 * If the comparison is anything other than equal, we're done. If they
+	 * compare equal though, we still have to consider whether the boundaries
+	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
+	 * two.
+	 */
+	if (cmpval == 0 && lower1 != lower2)
+		cmpval = lower1 ? 1 : -1;
+
+	return cmpval;
+}
+
+/* Used when sorting range bounds across all range partitions */
+static int32
+qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
+{
+	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
+	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
+	PartitionKey key = (PartitionKey) arg;
+
+	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
+								key->partcollation, b1->datums, b1->kind,
+								b1->lower, b2);
+}
+
+/*
+ * partition_list_bsearch
+ *		Returns the index of the greatest bound datum that is less than equal
+ * 		to the given value or -1 if all of the bound datums are greater
+ *
+ * *is_equal is set to true if the bound datum at the returned index is equal
+ * to the input value.
+ */
+static int
+partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+												 partcollation[0],
+												 boundinfo->datums[mid][0],
+												 value));
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_rbound_datum_cmp
+ *
+ * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
+ * is <, =, or > partition key of tuple (tuple_datums)
+ *
+ * n_tuple_datums, partsupfunc and partcollation give number of attributes in
+ * the bounds to be compared, comparison function to be used and the collations
+ * of attributes resp.
+ *
+ */
+static int32
+partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums)
+{
+	int			i;
+	int32		cmpval = -1;
+
+	for (i = 0; i < n_tuple_datums; i++)
+	{
+		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
+			return -1;
+		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
+			return 1;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
+												 rb_datums[i],
+												 tuple_datums[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	return cmpval;
+}
+
+/*
+ * partition_range_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given range bound or -1 if all of the range bounds are
+ *		greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input range bound
+ */
+static int
+partition_range_bsearch(int partnatts,
+						FmgrInfo *partsupfunc, Oid *partcollation,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_cmp(partnatts,
+									  partsupfunc, partcollation,
+									  boundinfo->datums[mid],
+									  boundinfo->kind[mid],
+									  (boundinfo->indexes[mid] == -1),
+									  probe);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_range_datum_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given tuple or -1 if all of the range bounds are greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input tuple.
+ */
+static int
+partition_range_datum_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_datum_cmp(partsupfunc, partcollation,
+											boundinfo->datums[mid],
+											boundinfo->kind[mid],
+											values,
+											nvalues);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_hash_bsearch
+ *		Returns the index of the greatest (modulus, remainder) pair that is
+ *		less than or equal to the given (modulus, remainder) pair or -1 if
+ *		all of them are greater
+ */
+static int
+partition_hash_bsearch(PartitionBoundInfo boundinfo,
+					   int modulus, int remainder)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval,
+					bound_modulus,
+					bound_remainder;
+
+		mid = (lo + hi + 1) / 2;
+		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
+		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
+		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
+									  modulus, remainder);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+
+			if (cmpval == 0)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * get_partition_bound_num_indexes
+ *
+ * Returns the number of the entries in the partition bound indexes array.
+ */
+static int
+get_partition_bound_num_indexes(PartitionBoundInfo bound)
+{
+	int			num_indexes;
+
+	Assert(bound);
+
+	switch (bound->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * The number of the entries in the indexes array is same as the
+			 * greatest modulus.
+			 */
+			num_indexes = get_greatest_modulus(bound);
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			num_indexes = bound->ndatums;
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			/* Range partitioned table has an extra index. */
+			num_indexes = bound->ndatums + 1;
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) bound->strategy);
+	}
+
+	return num_indexes;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 6ab4db26bd..e16a9b8b66 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -42,7 +42,6 @@
 #include "catalog/index.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_amproc.h"
 #include "catalog/pg_attrdef.h"
@@ -81,6 +80,7 @@
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partcache.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -261,7 +261,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
 static Relation AllocateRelationDesc(Form_pg_class relp);
 static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
 static void RelationBuildTupleDesc(Relation relation);
-static void RelationBuildPartitionKey(Relation relation);
 static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
 static void RelationInitPhysicalAddr(Relation relation);
 static void load_critical_index(Oid indexoid, Oid heapoid);
@@ -809,210 +808,6 @@ RelationBuildRuleLock(Relation relation)
 }
 
 /*
- * RelationBuildPartitionKey
- *		Build and attach to relcache partition key data of relation
- *
- * Partitioning key data is a complex structure; to avoid complicated logic to
- * free individual elements whenever the relcache entry is flushed, we give it
- * its own memory context, child of CacheMemoryContext, which can easily be
- * deleted on its own.  To avoid leaking memory in that context in case of an
- * error partway through this function, the context is initially created as a
- * child of CurTransactionContext and only re-parented to CacheMemoryContext
- * at the end, when no further errors are possible.  Also, we don't make this
- * context the current context except in very brief code sections, out of fear
- * that some of our callees allocate memory on their own which would be leaked
- * permanently.
- */
-static void
-RelationBuildPartitionKey(Relation relation)
-{
-	Form_pg_partitioned_table form;
-	HeapTuple	tuple;
-	bool		isnull;
-	int			i;
-	PartitionKey key;
-	AttrNumber *attrs;
-	oidvector  *opclass;
-	oidvector  *collation;
-	ListCell   *partexprs_item;
-	Datum		datum;
-	MemoryContext partkeycxt,
-				oldcxt;
-	int16		procnum;
-
-	tuple = SearchSysCache1(PARTRELID,
-							ObjectIdGetDatum(RelationGetRelid(relation)));
-
-	/*
-	 * The following happens when we have created our pg_class entry but not
-	 * the pg_partitioned_table entry yet.
-	 */
-	if (!HeapTupleIsValid(tuple))
-		return;
-
-	partkeycxt = AllocSetContextCreateExtended(CurTransactionContext,
-											   RelationGetRelationName(relation),
-											   MEMCONTEXT_COPY_NAME,
-											   ALLOCSET_SMALL_SIZES);
-
-	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
-												sizeof(PartitionKeyData));
-
-	/* Fixed-length attributes */
-	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	key->strategy = form->partstrat;
-	key->partnatts = form->partnatts;
-
-	/*
-	 * We can rely on the first variable-length attribute being mapped to the
-	 * relevant field of the catalog's C struct, because all previous
-	 * attributes are non-nullable and fixed-length.
-	 */
-	attrs = form->partattrs.values;
-
-	/* But use the hard way to retrieve further variable-length attributes */
-	/* Operator class */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partclass, &isnull);
-	Assert(!isnull);
-	opclass = (oidvector *) DatumGetPointer(datum);
-
-	/* Collation */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partcollation, &isnull);
-	Assert(!isnull);
-	collation = (oidvector *) DatumGetPointer(datum);
-
-	/* Expressions */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partexprs, &isnull);
-	if (!isnull)
-	{
-		char	   *exprString;
-		Node	   *expr;
-
-		exprString = TextDatumGetCString(datum);
-		expr = stringToNode(exprString);
-		pfree(exprString);
-
-		/*
-		 * Run the expressions through const-simplification since the planner
-		 * will be comparing them to similarly-processed qual clause operands,
-		 * and may fail to detect valid matches without this step; fix
-		 * opfuncids while at it.  We don't need to bother with
-		 * canonicalize_qual() though, because partition expressions should be
-		 * in canonical form already (ie, no need for OR-merging or constant
-		 * elimination).
-		 */
-		expr = eval_const_expressions(NULL, expr);
-		fix_opfuncids(expr);
-
-		oldcxt = MemoryContextSwitchTo(partkeycxt);
-		key->partexprs = (List *) copyObject(expr);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
-	oldcxt = MemoryContextSwitchTo(partkeycxt);
-	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
-	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
-
-	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-
-	/* Gather type and collation info as well */
-	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
-	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
-	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
-	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
-	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	MemoryContextSwitchTo(oldcxt);
-
-	/* determine support function number to search for */
-	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
-		HASHEXTENDED_PROC : BTORDER_PROC;
-
-	/* Copy partattrs and fill other per-attribute info */
-	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		AttrNumber	attno = key->partattrs[i];
-		HeapTuple	opclasstup;
-		Form_pg_opclass opclassform;
-		Oid			funcid;
-
-		/* Collect opfamily information */
-		opclasstup = SearchSysCache1(CLAOID,
-									 ObjectIdGetDatum(opclass->values[i]));
-		if (!HeapTupleIsValid(opclasstup))
-			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
-
-		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
-		key->partopfamily[i] = opclassform->opcfamily;
-		key->partopcintype[i] = opclassform->opcintype;
-
-		/* Get a support function for the specified opfamily and datatypes */
-		funcid = get_opfamily_proc(opclassform->opcfamily,
-								   opclassform->opcintype,
-								   opclassform->opcintype,
-								   procnum);
-		if (!OidIsValid(funcid))
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
-							NameStr(opclassform->opcname),
-							(key->strategy == PARTITION_STRATEGY_HASH) ?
-							"hash" : "btree",
-							procnum,
-							format_type_be(opclassform->opcintype))));
-
-		fmgr_info(funcid, &key->partsupfunc[i]);
-
-		/* Collation */
-		key->partcollation[i] = collation->values[i];
-
-		/* Collect type information */
-		if (attno != 0)
-		{
-			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
-
-			key->parttypid[i] = att->atttypid;
-			key->parttypmod[i] = att->atttypmod;
-			key->parttypcoll[i] = att->attcollation;
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			key->parttypid[i] = exprType(lfirst(partexprs_item));
-			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
-			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
-
-			partexprs_item = lnext(partexprs_item);
-		}
-		get_typlenbyvalalign(key->parttypid[i],
-							 &key->parttyplen[i],
-							 &key->parttypbyval[i],
-							 &key->parttypalign[i]);
-
-		ReleaseSysCache(opclasstup);
-	}
-
-	ReleaseSysCache(tuple);
-
-	/*
-	 * Success --- reparent our context and make the relcache point to the
-	 * newly constructed key
-	 */
-	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
-	relation->rd_partkeycxt = partkeycxt;
-	relation->rd_partkey = key;
-}
-
-/*
  *		equalRuleLocks
  *
  *		Determine whether two RuleLocks are equivalent
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2faf0ca26e..21abc8babd 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -14,63 +14,23 @@
 #define PARTITION_H
 
 #include "fmgr.h"
-#include "executor/tuptable.h"
-#include "nodes/execnodes.h"
-#include "parser/parse_node.h"
+#include "nodes/parsenodes.h"
 #include "utils/rel.h"
 
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-/*
- * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
- * associated with partitioned tables as part of its partition descriptor.
- *
- * The internal structure is opaque outside partition.c.
- */
-typedef struct PartitionBoundInfoData *PartitionBoundInfo;
-
-/*
- * Information about partitions of a partitioned table.
- */
-typedef struct PartitionDescData
-{
-	int			nparts;			/* Number of partitions */
-	Oid		   *oids;			/* OIDs of partitions */
-	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
-} PartitionDescData;
-
-typedef struct PartitionDescData *PartitionDesc;
-
-extern void RelationBuildPartitionDesc(Relation relation);
-extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
-					   bool *parttypbyval, PartitionBoundInfo b1,
-					   PartitionBoundInfo b2);
-extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key);
-
-extern void check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec);
 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 fromrel_varno,
 						Relation to_rel, Relation from_rel,
 						bool *found_whole_row);
-extern List *RelationGetPartitionQual(Relation rel);
-extern Expr *get_partition_qual_relid(Oid relid);
-extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr);
 
-extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
 extern Oid	get_default_partition_oid(Oid parentId);
 extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
-extern int get_partition_for_tuple(Relation relation, Datum *values,
-						bool *isnull);
-
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 03a599ad57..bd995cb2da 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -13,10 +13,10 @@
 #ifndef EXECPARTITION_H
 #define EXECPARTITION_H
 
-#include "catalog/partition.h"
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "utils/partcache.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
diff --git a/src/include/utils/partcache.h b/src/include/utils/partcache.h
new file mode 100644
index 0000000000..4bbd2962a9
--- /dev/null
+++ b/src/include/utils/partcache.h
@@ -0,0 +1,187 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.h
+ *		Header file for partitioning related cached data structures and
+ *		manipulation functions
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partcache.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTCACHE_H
+#define PARTCACHE_H
+
+#include "fmgr.h"
+#include "utils/relcache.h"
+
+/*
+ * Information about the partition key of a relation
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partitioning strategy */
+	int16		partnatts;		/* number of columns in the partition key */
+	AttrNumber *partattrs;		/* attribute numbers of columns in the
+								 * partition key */
+	List	   *partexprs;		/* list of expressions in the partitioning
+								 * key, or NIL */
+
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+
+	/* Partitioning collation per attribute */
+	Oid		   *partcollation;
+
+	/* Type information per attribute */
+	Oid		   *parttypid;
+	int32	   *parttypmod;
+	int16	   *parttyplen;
+	bool	   *parttypbyval;
+	char	   *parttypalign;
+	Oid		   *parttypcoll;
+}			PartitionKeyData;
+
+typedef struct PartitionKeyData *PartitionKey;
+
+typedef struct PartitionBoundInfoData *PartitionBoundInfo;
+
+/*
+ * Information about partitions of a partitioned table.
+ */
+typedef struct PartitionDescData
+{
+	int			nparts;			/* Number of partitions */
+	Oid		   *oids;			/* OIDs of partitions */
+	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
+} PartitionDescData;
+
+typedef struct PartitionDescData *PartitionDesc;
+
+/*
+ * Information about bounds of a partitioned relation
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+	char		strategy;		/* hash, list or range? */
+	int			ndatums;		/* Length of the datums following array */
+	Datum	  **datums;
+	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+									 * NULL for hash and list partitioned
+									 * tables */
+	int		   *indexes;		/* Partition indexes */
+	int			null_index;		/* Index of the null-accepting partition; -1
+								 * if there isn't one */
+	int			default_index;	/* Index of the default partition; -1 if there
+								 * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * PartitionKey inquiry functions
+ */
+static inline int
+get_partition_strategy(PartitionKey key)
+{
+	return key->strategy;
+}
+
+static inline int
+get_partition_natts(PartitionKey key)
+{
+	return key->partnatts;
+}
+
+static inline List *
+get_partition_exprs(PartitionKey key)
+{
+	return key->partexprs;
+}
+
+/*
+ * PartitionKey inquiry functions - one column
+ */
+static inline int16
+get_partition_col_attnum(PartitionKey key, int col)
+{
+	return key->partattrs[col];
+}
+
+static inline Oid
+get_partition_col_typid(PartitionKey key, int col)
+{
+	return key->parttypid[col];
+}
+
+static inline int32
+get_partition_col_typmod(PartitionKey key, int col)
+{
+	return key->parttypmod[col];
+}
+
+extern void RelationBuildPartitionKey(Relation relation);
+extern void RelationBuildPartitionDesc(Relation relation);
+extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
+					   bool *parttypbyval, PartitionBoundInfo b1,
+					   PartitionBoundInfo b2);
+
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
+
+extern void check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec);
+
+extern List *RelationGetPartitionQual(Relation rel);
+extern Expr *get_partition_qual_relid(Oid relid);
+
+extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr);
+
+extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+
+extern int get_greatest_modulus(PartitionBoundInfo b);
+extern uint64 compute_hash_value(PartitionKey key, Datum *values,
+						bool *isnull);
+
+/* For tuple routing */
+extern int get_partition_for_tuple(Relation relation, Datum *values,
+						bool *isnull);
+
+#endif							/* PARTCACHE_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index aa8add544a..db3545ec7f 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -19,12 +19,11 @@
 #include "catalog/pg_class.h"
 #include "catalog/pg_index.h"
 #include "catalog/pg_publication.h"
-#include "fmgr.h"
 #include "nodes/bitmapset.h"
 #include "rewrite/prs2lock.h"
 #include "storage/block.h"
 #include "storage/relfilenode.h"
-#include "utils/relcache.h"
+#include "utils/partcache.h"
 #include "utils/reltrigger.h"
 
 
@@ -47,36 +46,6 @@ typedef struct LockInfoData
 typedef LockInfoData *LockInfo;
 
 /*
- * Information about the partition key of a relation
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partitioning strategy */
-	int16		partnatts;		/* number of columns in the partition key */
-	AttrNumber *partattrs;		/* attribute numbers of columns in the
-								 * partition key */
-	List	   *partexprs;		/* list of expressions in the partitioning
-								 * key, or NIL */
-
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-
-	/* Partitioning collation per attribute */
-	Oid		   *partcollation;
-
-	/* Type information per attribute */
-	Oid		   *parttypid;
-	int32	   *parttypmod;
-	int16	   *parttyplen;
-	bool	   *parttypbyval;
-	char	   *parttypalign;
-	Oid		   *parttypcoll;
-}			PartitionKeyData;
-
-typedef struct PartitionKeyData *PartitionKey;
-
-/*
  * Here are the contents of a relation cache entry.
  */
 
@@ -593,48 +562,6 @@ typedef struct ViewOptions
 #define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
 
 /*
- * PartitionKey inquiry functions
- */
-static inline int
-get_partition_strategy(PartitionKey key)
-{
-	return key->strategy;
-}
-
-static inline int
-get_partition_natts(PartitionKey key)
-{
-	return key->partnatts;
-}
-
-static inline List *
-get_partition_exprs(PartitionKey key)
-{
-	return key->partexprs;
-}
-
-/*
- * PartitionKey inquiry functions - one column
- */
-static inline int16
-get_partition_col_attnum(PartitionKey key, int col)
-{
-	return key->partattrs[col];
-}
-
-static inline Oid
-get_partition_col_typid(PartitionKey key, int col)
-{
-	return key->parttypid[col];
-}
-
-static inline int32
-get_partition_col_typmod(PartitionKey key, int col)
-{
-	return key->parttypmod[col];
-}
-
-/*
  * RelationGetPartitionDesc
  *		Returns partition descriptor for a relation.
  */
-- 
2.11.0

#17Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#14)
Re: Re: Re: reorganizing partitioning code

On Wed, Mar 21, 2018 at 10:20 AM, Alvaro Herrera
<alvherre@alvh.no-ip.org> wrote:

Let's keep this entry open till the last minute.

Ugh, I don't like keeping things open till the last minute all that
much, especially if they're not being updated. But since this has
been updated I guess it's somewhat moot.

Are you going to pick this up RSN?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#18Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Robert Haas (#17)
Re: Re: Re: reorganizing partitioning code

Robert Haas wrote:

On Wed, Mar 21, 2018 at 10:20 AM, Alvaro Herrera
<alvherre@alvh.no-ip.org> wrote:

Let's keep this entry open till the last minute.

Ugh, I don't like keeping things open till the last minute all that
much, especially if they're not being updated. But since this has
been updated I guess it's somewhat moot.

Are you going to pick this up RSN?

If during next week qualifies as RSN, then yes.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#19Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#16)
1 attachment(s)
Re: reorganizing partitioning code

On 2018/03/22 11:45, Amit Langote wrote:

FWIW, I did manage to rebase it this morning and posting it here.

Rebased again.

I started wondering if we should separate out stuff related to partition
bounds. That is create a utils/partbound.h and put PartitionBoundInfo and
related comparison and search functions into a utils/adt/partbound.c. I
had started thinking about that when I looked at the code added by the
patch submitted on the "advanced partition matching algorithm for
partition-wise join" thread [1]https://commitfest.postgresql.org/17/1553/. I haven't done anything about that though.

Thanks,
Amit

[1]: https://commitfest.postgresql.org/17/1553/

Attachments:

v6-0001-Reorganize-partitioning-code.patchtext/plain; charset=UTF-8; name=v6-0001-Reorganize-partitioning-code.patchDownload
From 78ee1d584a67ac5ab1193ca661b74be16d3db68d Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Tue, 13 Feb 2018 15:59:30 +0900
Subject: [PATCH v6] Reorganize partitioning code

---
 src/backend/catalog/partition.c        | 2657 +++++---------------------------
 src/backend/executor/execMain.c        |    1 -
 src/backend/executor/execPartition.c   |    1 +
 src/backend/optimizer/path/joinrels.c  |    2 +-
 src/backend/optimizer/prep/prepunion.c |    1 -
 src/backend/optimizer/util/plancat.c   |    1 -
 src/backend/optimizer/util/relnode.c   |    2 +-
 src/backend/utils/adt/ruleutils.c      |    1 -
 src/backend/utils/cache/Makefile       |    6 +-
 src/backend/utils/cache/partcache.c    | 2151 ++++++++++++++++++++++++++
 src/backend/utils/cache/relcache.c     |  208 +--
 src/include/catalog/partition.h        |   42 +-
 src/include/executor/execPartition.h   |    2 +-
 src/include/utils/partcache.h          |  187 +++
 src/include/utils/rel.h                |   75 +-
 15 files changed, 2698 insertions(+), 2639 deletions(-)
 create mode 100644 src/backend/utils/cache/partcache.c
 create mode 100644 src/include/utils/partcache.h

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 39ee773d93..9bb60820f8 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,11 +15,7 @@
 
 #include "postgres.h"
 
-#include "access/hash.h"
-#include "access/heapam.h"
 #include "access/htup_details.h"
-#include "access/nbtree.h"
-#include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
@@ -35,7 +31,6 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
-#include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
 #include "optimizer/prep.h"
@@ -48,106 +43,15 @@
 #include "utils/datum.h"
 #include "utils/fmgroids.h"
 #include "utils/hashutils.h"
-#include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
-#include "utils/ruleutils.h"
+#include "utils/snapmgr.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-	char		strategy;		/* hash, list or range? */
-	int			ndatums;		/* Length of the datums following array */
-	Datum	  **datums;
-	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-									 * NULL for hash and list partitioned
-									 * tables */
-	int		   *indexes;		/* Partition indexes */
-	int			null_index;		/* Index of the null-accepting partition; -1
-								 * if there isn't one */
-	int			default_index;	/* Index of the default partition; -1 if there
-								 * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-	int			modulus;
-	int			remainder;
-	int			index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-	int			index;
-	Datum		value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-	int			index;
-	Datum	   *datums;			/* range bound datums */
-	PartitionRangeDatumKind *kind;	/* the kind of each datum */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
-
 static Oid	get_partition_parent_worker(Relation inhRel, Oid relid);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
 							   List **ancestors);
-static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
-static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
-							   void *arg);
-static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
-						   void *arg);
-
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
@@ -163,1071 +67,6 @@ static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 				   bool for_default);
 static List *get_range_nulltest(PartitionKey key);
-static List *generate_partition_qual(Relation rel);
-
-static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
-					 List *datums, bool lower);
-static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
-					 int remainder2);
-static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
-					 Oid *partcollation, Datum *datums1,
-					 PartitionRangeDatumKind *kind1, bool lower1,
-					 PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
-						   Oid *partcollation,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal);
-static int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
-						Oid *partcollation,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
-							  Oid *partcollation,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionBoundInfo boundinfo,
-					   int modulus, int remainder);
-
-static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int	get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
-								 Datum *values, bool *isnull);
-
-/*
- * RelationBuildPartitionDesc
- *		Form rel's partition descriptor
- *
- * Not flushed from the cache by RelationClearRelation() unless changed because
- * of addition or removal of partition.
- */
-void
-RelationBuildPartitionDesc(Relation rel)
-{
-	List	   *inhoids,
-			   *partoids;
-	Oid		   *oids = NULL;
-	List	   *boundspecs = NIL;
-	ListCell   *cell;
-	int			i,
-				nparts;
-	PartitionKey key = RelationGetPartitionKey(rel);
-	PartitionDesc result;
-	MemoryContext oldcxt;
-
-	int			ndatums = 0;
-	int			default_index = -1;
-
-	/* Hash partitioning specific */
-	PartitionHashBound **hbounds = NULL;
-
-	/* List partitioning specific */
-	PartitionListValue **all_values = NULL;
-	int			null_index = -1;
-
-	/* Range partitioning specific */
-	PartitionRangeBound **rbounds = NULL;
-
-	/* Get partition oids from pg_inherits */
-	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
-
-	/* Collect bound spec nodes in a list */
-	i = 0;
-	partoids = NIL;
-	foreach(cell, inhoids)
-	{
-		Oid			inhrelid = lfirst_oid(cell);
-		HeapTuple	tuple;
-		Datum		datum;
-		bool		isnull;
-		Node	   *boundspec;
-
-		tuple = SearchSysCache1(RELOID, inhrelid);
-		if (!HeapTupleIsValid(tuple))
-			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
-
-		/*
-		 * It is possible that the pg_class tuple of a partition has not been
-		 * updated yet to set its relpartbound field.  The only case where
-		 * this happens is when we open the parent relation to check using its
-		 * partition descriptor that a new partition's bound does not overlap
-		 * some existing partition.
-		 */
-		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
-		{
-			ReleaseSysCache(tuple);
-			continue;
-		}
-
-		datum = SysCacheGetAttr(RELOID, tuple,
-								Anum_pg_class_relpartbound,
-								&isnull);
-		Assert(!isnull);
-		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
-
-		/*
-		 * Sanity check: If the PartitionBoundSpec says this is the default
-		 * partition, its OID should correspond to whatever's stored in
-		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
-		 */
-		if (castNode(PartitionBoundSpec, boundspec)->is_default)
-		{
-			Oid			partdefid;
-
-			partdefid = get_default_partition_oid(RelationGetRelid(rel));
-			if (partdefid != inhrelid)
-				elog(ERROR, "expected partdefid %u, but got %u",
-					 inhrelid, partdefid);
-		}
-
-		boundspecs = lappend(boundspecs, boundspec);
-		partoids = lappend_oid(partoids, inhrelid);
-		ReleaseSysCache(tuple);
-	}
-
-	nparts = list_length(partoids);
-
-	if (nparts > 0)
-	{
-		oids = (Oid *) palloc(nparts * sizeof(Oid));
-		i = 0;
-		foreach(cell, partoids)
-			oids[i++] = lfirst_oid(cell);
-
-		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_HASH)
-		{
-			ndatums = nparts;
-			hbounds = (PartitionHashBound **)
-				palloc(nparts * sizeof(PartitionHashBound *));
-
-			i = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-
-				if (spec->strategy != PARTITION_STRATEGY_HASH)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				hbounds[i] = (PartitionHashBound *)
-					palloc(sizeof(PartitionHashBound));
-
-				hbounds[i]->modulus = spec->modulus;
-				hbounds[i]->remainder = spec->remainder;
-				hbounds[i]->index = i;
-				i++;
-			}
-
-			/* Sort all the bounds in ascending order */
-			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
-				  qsort_partition_hbound_cmp);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_LIST)
-		{
-			List	   *non_null_values = NIL;
-
-			/*
-			 * Create a unified list of non-null values across all partitions.
-			 */
-			i = 0;
-			null_index = -1;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				ListCell   *c;
-
-				if (spec->strategy != PARTITION_STRATEGY_LIST)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the list of non-null
-				 * datums for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i;
-					i++;
-					continue;
-				}
-
-				foreach(c, spec->listdatums)
-				{
-					Const	   *val = castNode(Const, lfirst(c));
-					PartitionListValue *list_value = NULL;
-
-					if (!val->constisnull)
-					{
-						list_value = (PartitionListValue *)
-							palloc0(sizeof(PartitionListValue));
-						list_value->index = i;
-						list_value->value = val->constvalue;
-					}
-					else
-					{
-						/*
-						 * Never put a null into the values array, flag
-						 * instead for the code further down below where we
-						 * construct the actual relcache struct.
-						 */
-						if (null_index != -1)
-							elog(ERROR, "found null more than once");
-						null_index = i;
-					}
-
-					if (list_value)
-						non_null_values = lappend(non_null_values,
-												  list_value);
-				}
-
-				i++;
-			}
-
-			ndatums = list_length(non_null_values);
-
-			/*
-			 * Collect all list values in one array. Alongside the value, we
-			 * also save the index of partition the value comes from.
-			 */
-			all_values = (PartitionListValue **) palloc(ndatums *
-														sizeof(PartitionListValue *));
-			i = 0;
-			foreach(cell, non_null_values)
-			{
-				PartitionListValue *src = lfirst(cell);
-
-				all_values[i] = (PartitionListValue *)
-					palloc(sizeof(PartitionListValue));
-				all_values[i]->value = src->value;
-				all_values[i]->index = src->index;
-				i++;
-			}
-
-			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
-					  qsort_partition_list_value_cmp, (void *) key);
-		}
-		else if (key->strategy == PARTITION_STRATEGY_RANGE)
-		{
-			int			k;
-			PartitionRangeBound **all_bounds,
-					   *prev;
-
-			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
-														  sizeof(PartitionRangeBound *));
-
-			/*
-			 * Create a unified list of range bounds across all the
-			 * partitions.
-			 */
-			i = ndatums = 0;
-			foreach(cell, boundspecs)
-			{
-				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
-													lfirst(cell));
-				PartitionRangeBound *lower,
-						   *upper;
-
-				if (spec->strategy != PARTITION_STRATEGY_RANGE)
-					elog(ERROR, "invalid strategy in partition bound spec");
-
-				/*
-				 * Note the index of the partition bound spec for the default
-				 * partition. There's no datum to add to the allbounds array
-				 * for this partition.
-				 */
-				if (spec->is_default)
-				{
-					default_index = i++;
-					continue;
-				}
-
-				lower = make_one_range_bound(key, i, spec->lowerdatums,
-											 true);
-				upper = make_one_range_bound(key, i, spec->upperdatums,
-											 false);
-				all_bounds[ndatums++] = lower;
-				all_bounds[ndatums++] = upper;
-				i++;
-			}
-
-			Assert(ndatums == nparts * 2 ||
-				   (default_index != -1 && ndatums == (nparts - 1) * 2));
-
-			/* Sort all the bounds in ascending order */
-			qsort_arg(all_bounds, ndatums,
-					  sizeof(PartitionRangeBound *),
-					  qsort_partition_rbound_cmp,
-					  (void *) key);
-
-			/* Save distinct bounds from all_bounds into rbounds. */
-			rbounds = (PartitionRangeBound **)
-				palloc(ndatums * sizeof(PartitionRangeBound *));
-			k = 0;
-			prev = NULL;
-			for (i = 0; i < ndatums; i++)
-			{
-				PartitionRangeBound *cur = all_bounds[i];
-				bool		is_distinct = false;
-				int			j;
-
-				/* Is the current bound distinct from the previous one? */
-				for (j = 0; j < key->partnatts; j++)
-				{
-					Datum		cmpval;
-
-					if (prev == NULL || cur->kind[j] != prev->kind[j])
-					{
-						is_distinct = true;
-						break;
-					}
-
-					/*
-					 * If the bounds are both MINVALUE or MAXVALUE, stop now
-					 * and treat them as equal, since any values after this
-					 * point must be ignored.
-					 */
-					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
-						break;
-
-					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
-											   key->partcollation[j],
-											   cur->datums[j],
-											   prev->datums[j]);
-					if (DatumGetInt32(cmpval) != 0)
-					{
-						is_distinct = true;
-						break;
-					}
-				}
-
-				/*
-				 * Only if the bound is distinct save it into a temporary
-				 * array i.e. rbounds which is later copied into boundinfo
-				 * datums array.
-				 */
-				if (is_distinct)
-					rbounds[k++] = all_bounds[i];
-
-				prev = cur;
-			}
-
-			/* Update ndatums to hold the count of distinct datums. */
-			ndatums = k;
-		}
-		else
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/* Now build the actual relcache partition descriptor */
-	rel->rd_pdcxt = AllocSetContextCreate(CacheMemoryContext,
-										  "partition descriptor",
-										  ALLOCSET_DEFAULT_SIZES);
-	MemoryContextCopySetIdentifier(rel->rd_pdcxt, RelationGetRelationName(rel));
-
-	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
-
-	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
-	result->nparts = nparts;
-	if (nparts > 0)
-	{
-		PartitionBoundInfo boundinfo;
-		int		   *mapping;
-		int			next_index = 0;
-
-		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
-
-		boundinfo = (PartitionBoundInfoData *)
-			palloc0(sizeof(PartitionBoundInfoData));
-		boundinfo->strategy = key->strategy;
-		boundinfo->default_index = -1;
-		boundinfo->ndatums = ndatums;
-		boundinfo->null_index = -1;
-		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
-
-		/* Initialize mapping array with invalid values */
-		mapping = (int *) palloc(sizeof(int) * nparts);
-		for (i = 0; i < nparts; i++)
-			mapping[i] = -1;
-
-		switch (key->strategy)
-		{
-			case PARTITION_STRATEGY_HASH:
-				{
-					/* Modulus are stored in ascending order */
-					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
-
-					boundinfo->indexes = (int *) palloc(greatest_modulus *
-														sizeof(int));
-
-					for (i = 0; i < greatest_modulus; i++)
-						boundinfo->indexes[i] = -1;
-
-					for (i = 0; i < nparts; i++)
-					{
-						int			modulus = hbounds[i]->modulus;
-						int			remainder = hbounds[i]->remainder;
-
-						boundinfo->datums[i] = (Datum *) palloc(2 *
-																sizeof(Datum));
-						boundinfo->datums[i][0] = Int32GetDatum(modulus);
-						boundinfo->datums[i][1] = Int32GetDatum(remainder);
-
-						while (remainder < greatest_modulus)
-						{
-							/* overlap? */
-							Assert(boundinfo->indexes[remainder] == -1);
-							boundinfo->indexes[remainder] = i;
-							remainder += modulus;
-						}
-
-						mapping[hbounds[i]->index] = i;
-						pfree(hbounds[i]);
-					}
-					pfree(hbounds);
-					break;
-				}
-
-			case PARTITION_STRATEGY_LIST:
-				{
-					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
-
-					/*
-					 * Copy values.  Indexes of individual values are mapped
-					 * to canonical values so that they match for any two list
-					 * partitioned tables with same number of partitions and
-					 * same lists per partition.  One way to canonicalize is
-					 * to assign the index in all_values[] of the smallest
-					 * value of each partition, as the index of all of the
-					 * partition's values.
-					 */
-					for (i = 0; i < ndatums; i++)
-					{
-						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
-						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
-															key->parttypbyval[0],
-															key->parttyplen[0]);
-
-						/* If the old index has no mapping, assign one */
-						if (mapping[all_values[i]->index] == -1)
-							mapping[all_values[i]->index] = next_index++;
-
-						boundinfo->indexes[i] = mapping[all_values[i]->index];
-					}
-
-					/*
-					 * If null-accepting partition has no mapped index yet,
-					 * assign one.  This could happen if such partition
-					 * accepts only null and hence not covered in the above
-					 * loop which only handled non-null values.
-					 */
-					if (null_index != -1)
-					{
-						Assert(null_index >= 0);
-						if (mapping[null_index] == -1)
-							mapping[null_index] = next_index++;
-						boundinfo->null_index = mapping[null_index];
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						/*
-						 * The default partition accepts any value not
-						 * specified in the lists of other partitions, hence
-						 * it should not get mapped index while assigning
-						 * those for non-null datums.
-						 */
-						Assert(default_index >= 0 &&
-							   mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-
-					/* All partition must now have a valid mapping */
-					Assert(next_index == nparts);
-					break;
-				}
-
-			case PARTITION_STRATEGY_RANGE:
-				{
-					boundinfo->kind = (PartitionRangeDatumKind **)
-						palloc(ndatums *
-							   sizeof(PartitionRangeDatumKind *));
-					boundinfo->indexes = (int *) palloc((ndatums + 1) *
-														sizeof(int));
-
-					for (i = 0; i < ndatums; i++)
-					{
-						int			j;
-
-						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
-																sizeof(Datum));
-						boundinfo->kind[i] = (PartitionRangeDatumKind *)
-							palloc(key->partnatts *
-								   sizeof(PartitionRangeDatumKind));
-						for (j = 0; j < key->partnatts; j++)
-						{
-							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
-								boundinfo->datums[i][j] =
-									datumCopy(rbounds[i]->datums[j],
-											  key->parttypbyval[j],
-											  key->parttyplen[j]);
-							boundinfo->kind[i][j] = rbounds[i]->kind[j];
-						}
-
-						/*
-						 * There is no mapping for invalid indexes.
-						 *
-						 * Any lower bounds in the rbounds array have invalid
-						 * indexes assigned, because the values between the
-						 * previous bound (if there is one) and this (lower)
-						 * bound are not part of the range of any existing
-						 * partition.
-						 */
-						if (rbounds[i]->lower)
-							boundinfo->indexes[i] = -1;
-						else
-						{
-							int			orig_index = rbounds[i]->index;
-
-							/* If the old index has no mapping, assign one */
-							if (mapping[orig_index] == -1)
-								mapping[orig_index] = next_index++;
-
-							boundinfo->indexes[i] = mapping[orig_index];
-						}
-					}
-
-					/* Assign mapped index for the default partition. */
-					if (default_index != -1)
-					{
-						Assert(default_index >= 0 && mapping[default_index] == -1);
-						mapping[default_index] = next_index++;
-						boundinfo->default_index = mapping[default_index];
-					}
-					boundinfo->indexes[i] = -1;
-					break;
-				}
-
-			default:
-				elog(ERROR, "unexpected partition strategy: %d",
-					 (int) key->strategy);
-		}
-
-		result->boundinfo = boundinfo;
-
-		/*
-		 * Now assign OIDs from the original array into mapped indexes of the
-		 * result array.  Order of OIDs in the former is defined by the
-		 * catalog scan that retrieved them, whereas that in the latter is
-		 * defined by canonicalized representation of the partition bounds.
-		 */
-		for (i = 0; i < nparts; i++)
-			result->oids[mapping[i]] = oids[i];
-		pfree(mapping);
-	}
-
-	MemoryContextSwitchTo(oldcxt);
-	rel->rd_partdesc = result;
-}
-
-/*
- * Are two partition bound collections logically equal?
- *
- * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
- * This is also useful when b1 and b2 are bound collections of two separate
- * relations, respectively, because PartitionBoundInfo is a canonical
- * representation of partition bounds.
- */
-bool
-partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
-					   PartitionBoundInfo b1, PartitionBoundInfo b2)
-{
-	int			i;
-
-	if (b1->strategy != b2->strategy)
-		return false;
-
-	if (b1->ndatums != b2->ndatums)
-		return false;
-
-	if (b1->null_index != b2->null_index)
-		return false;
-
-	if (b1->default_index != b2->default_index)
-		return false;
-
-	if (b1->strategy == PARTITION_STRATEGY_HASH)
-	{
-		int			greatest_modulus = get_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_greatest_modulus(b2))
-			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their modulus
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
-	}
-	else
-	{
-		for (i = 0; i < b1->ndatums; i++)
-		{
-			int			j;
-
-			for (j = 0; j < partnatts; j++)
-			{
-				/* For range partitions, the bounds might not be finite. */
-				if (b1->kind != NULL)
-				{
-					/* The different kinds of bound all differ from each other */
-					if (b1->kind[i][j] != b2->kind[i][j])
-						return false;
-
-					/*
-					 * Non-finite bounds are equal without further
-					 * examination.
-					 */
-					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
-						continue;
-				}
-
-				/*
-				 * Compare the actual values. Note that it would be both
-				 * incorrect and unsafe to invoke the comparison operator
-				 * derived from the partitioning specification here.  It would
-				 * be incorrect because we want the relcache entry to be
-				 * updated for ANY change to the partition bounds, not just
-				 * those that the partitioning operator thinks are
-				 * significant.  It would be unsafe because we might reach
-				 * this code in the context of an aborted transaction, and an
-				 * arbitrary partitioning operator might not be safe in that
-				 * context.  datumIsEqual() should be simple enough to be
-				 * safe.
-				 */
-				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-								  parttypbyval[j], parttyplen[j]))
-					return false;
-			}
-
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-		}
-
-		/* There are ndatums+1 indexes in case of range partitions */
-		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
-			b1->indexes[i] != b2->indexes[i])
-			return false;
-	}
-	return true;
-}
-
-/*
- * Return a copy of given PartitionBoundInfo structure. The data types of bounds
- * are described by given partition key specification.
- */
-PartitionBoundInfo
-partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key)
-{
-	PartitionBoundInfo dest;
-	int			i;
-	int			ndatums;
-	int			partnatts;
-	int			num_indexes;
-
-	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
-
-	dest->strategy = src->strategy;
-	ndatums = dest->ndatums = src->ndatums;
-	partnatts = key->partnatts;
-
-	num_indexes = get_partition_bound_num_indexes(src);
-
-	/* List partitioned tables have only a single partition key. */
-	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
-
-	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
-
-	if (src->kind != NULL)
-	{
-		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
-														 sizeof(PartitionRangeDatumKind *));
-		for (i = 0; i < ndatums; i++)
-		{
-			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
-															   sizeof(PartitionRangeDatumKind));
-
-			memcpy(dest->kind[i], src->kind[i],
-				   sizeof(PartitionRangeDatumKind) * key->partnatts);
-		}
-	}
-	else
-		dest->kind = NULL;
-
-	for (i = 0; i < ndatums; i++)
-	{
-		int			j;
-
-		/*
-		 * For a corresponding to hash partition, datums array will have two
-		 * elements - modulus and remainder.
-		 */
-		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
-		int			natts = hash_part ? 2 : partnatts;
-
-		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
-
-		for (j = 0; j < natts; j++)
-		{
-			bool		byval;
-			int			typlen;
-
-			if (hash_part)
-			{
-				typlen = sizeof(int32); /* Always int4 */
-				byval = true;	/* int4 is pass-by-value */
-			}
-			else
-			{
-				byval = key->parttypbyval[j];
-				typlen = key->parttyplen[j];
-			}
-
-			if (dest->kind == NULL ||
-				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
-				dest->datums[i][j] = datumCopy(src->datums[i][j],
-											   byval, typlen);
-		}
-	}
-
-	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
-	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
-
-	dest->null_index = src->null_index;
-	dest->default_index = src->default_index;
-
-	return dest;
-}
-
-/*
- * check_new_partition_bound
- *
- * Checks if the new partition's bound overlaps any of the existing partitions
- * of parent.  Also performs additional checks as necessary per strategy.
- */
-void
-check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec)
-{
-	PartitionKey key = RelationGetPartitionKey(parent);
-	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	PartitionBoundInfo boundinfo = partdesc->boundinfo;
-	ParseState *pstate = make_parsestate(NULL);
-	int			with = -1;
-	bool		overlap = false;
-
-	if (spec->is_default)
-	{
-		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
-			return;
-
-		/* Default partition already exists, error out. */
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
-				 parser_errposition(pstate, spec->location)));
-	}
-
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
-				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					Datum	  **datums = boundinfo->datums;
-					int			ndatums = boundinfo->ndatums;
-					int			greatest_modulus;
-					int			remainder;
-					int			offset;
-					bool		valid_modulus = true;
-					int			prev_modulus,	/* Previous largest modulus */
-								next_modulus;	/* Next largest modulus */
-
-					/*
-					 * Check rule that every modulus must be a factor of the
-					 * next larger modulus.  For example, if you have a bunch
-					 * of partitions that all have modulus 5, you can add a
-					 * new partition with modulus 10 or a new partition with
-					 * modulus 15, but you cannot add both a partition with
-					 * modulus 10 and a partition with modulus 15, because 10
-					 * is not a factor of 15.
-					 *
-					 * Get the greatest (modulus, remainder) pair contained in
-					 * boundinfo->datums that is less than or equal to the
-					 * (spec->modulus, spec->remainder) pair.
-					 */
-					offset = partition_hash_bsearch(boundinfo,
-													spec->modulus,
-													spec->remainder);
-					if (offset < 0)
-					{
-						next_modulus = DatumGetInt32(datums[0][0]);
-						valid_modulus = (next_modulus % spec->modulus) == 0;
-					}
-					else
-					{
-						prev_modulus = DatumGetInt32(datums[offset][0]);
-						valid_modulus = (spec->modulus % prev_modulus) == 0;
-
-						if (valid_modulus && (offset + 1) < ndatums)
-						{
-							next_modulus = DatumGetInt32(datums[offset + 1][0]);
-							valid_modulus = (next_modulus % spec->modulus) == 0;
-						}
-					}
-
-					if (!valid_modulus)
-						ereport(ERROR,
-								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
-
-					greatest_modulus = get_greatest_modulus(boundinfo);
-					remainder = spec->remainder;
-
-					/*
-					 * Normally, the lowest remainder that could conflict with
-					 * the new partition is equal to the remainder specified
-					 * for the new partition, but when the new partition has a
-					 * modulus higher than any used so far, we need to adjust.
-					 */
-					if (remainder >= greatest_modulus)
-						remainder = remainder % greatest_modulus;
-
-					/* Check every potentially-conflicting remainder. */
-					do
-					{
-						if (boundinfo->indexes[remainder] != -1)
-						{
-							overlap = true;
-							with = boundinfo->indexes[remainder];
-							break;
-						}
-						remainder += spec->modulus;
-					} while (remainder < greatest_modulus);
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_LIST:
-			{
-				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-
-				if (partdesc->nparts > 0)
-				{
-					ListCell   *cell;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo) ||
-							partition_bound_has_default(boundinfo)));
-
-					foreach(cell, spec->listdatums)
-					{
-						Const	   *val = castNode(Const, lfirst(cell));
-
-						if (!val->constisnull)
-						{
-							int			offset;
-							bool		equal;
-
-							offset = partition_list_bsearch(key->partsupfunc,
-														key->partcollation,
-															boundinfo,
-															val->constvalue,
-															&equal);
-							if (offset >= 0 && equal)
-							{
-								overlap = true;
-								with = boundinfo->indexes[offset];
-								break;
-							}
-						}
-						else if (partition_bound_accepts_nulls(boundinfo))
-						{
-							overlap = true;
-							with = boundinfo->null_index;
-							break;
-						}
-					}
-				}
-
-				break;
-			}
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				PartitionRangeBound *lower,
-						   *upper;
-
-				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
-				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
-
-				/*
-				 * First check if the resulting range would be empty with
-				 * specified lower and upper bounds
-				 */
-				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
-										 key->partcollation, lower->datums,
-										 lower->kind, true, upper) >= 0)
-				{
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-							 errmsg("empty range bound specified for partition \"%s\"",
-									relname),
-							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
-									   get_range_partbound_string(spec->lowerdatums),
-									   get_range_partbound_string(spec->upperdatums)),
-							 parser_errposition(pstate, spec->location)));
-				}
-
-				if (partdesc->nparts > 0)
-				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
-					int			offset;
-					bool		equal;
-
-					Assert(boundinfo &&
-						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
-						   (boundinfo->ndatums > 0 ||
-							partition_bound_has_default(boundinfo)));
-
-					/*
-					 * Test whether the new lower bound (which is treated
-					 * inclusively as part of the new partition) lies inside
-					 * an existing partition, or in a gap.
-					 *
-					 * If it's inside an existing partition, the bound at
-					 * offset + 1 will be the upper bound of that partition,
-					 * and its index will be >= 0.
-					 *
-					 * If it's in a gap, the bound at offset + 1 will be the
-					 * lower bound of the next partition, and its index will
-					 * be -1. This is also true if there is no next partition,
-					 * since the index array is initialised with an extra -1
-					 * at the end.
-					 */
-					offset = partition_range_bsearch(key->partnatts,
-													 key->partsupfunc,
-													 key->partcollation,
-													 boundinfo, lower,
-													 &equal);
-
-					if (boundinfo->indexes[offset + 1] < 0)
-					{
-						/*
-						 * Check that the new partition will fit in the gap.
-						 * For it to fit, the new upper bound must be less
-						 * than or equal to the lower bound of the next
-						 * partition, if there is one.
-						 */
-						if (offset + 1 < boundinfo->ndatums)
-						{
-							int32		cmpval;
-							Datum 	   *datums;
-							PartitionRangeDatumKind *kind;
-							bool		is_lower;
-
-							datums = boundinfo->datums[offset + 1];
-							kind = boundinfo->kind[offset + 1];
-							is_lower = (boundinfo->indexes[offset + 1] == -1);
-
-							cmpval = partition_rbound_cmp(key->partnatts,
-														  key->partsupfunc,
-														  key->partcollation,
-														  datums, kind,
-														  is_lower, upper);
-							if (cmpval < 0)
-							{
-								/*
-								 * The new partition overlaps with the
-								 * existing partition between offset + 1 and
-								 * offset + 2.
-								 */
-								overlap = true;
-								with = boundinfo->indexes[offset + 2];
-							}
-						}
-					}
-					else
-					{
-						/*
-						 * The new partition overlaps with the existing
-						 * partition between offset and offset + 1.
-						 */
-						overlap = true;
-						with = boundinfo->indexes[offset + 1];
-					}
-				}
-
-				break;
-			}
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	if (overlap)
-	{
-		Assert(with >= 0);
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-				 errmsg("partition \"%s\" would overlap partition \"%s\"",
-						relname, get_rel_name(partdesc->oids[with])),
-				 parser_errposition(pstate, spec->location)));
-	}
-}
 
 /*
  * check_default_allows_bound
@@ -1409,43 +248,6 @@ get_partition_parent(Oid relid)
 }
 
 /*
- * get_partition_parent_worker
- *		Scan the pg_inherits relation to return the OID of the parent of the
- *		given relation
- */
-static Oid
-get_partition_parent_worker(Relation inhRel, Oid relid)
-{
-	SysScanDesc scan;
-	ScanKeyData key[2];
-	Oid			result = InvalidOid;
-	HeapTuple	tuple;
-
-	ScanKeyInit(&key[0],
-				Anum_pg_inherits_inhrelid,
-				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(relid));
-	ScanKeyInit(&key[1],
-				Anum_pg_inherits_inhseqno,
-				BTEqualStrategyNumber, F_INT4EQ,
-				Int32GetDatum(1));
-
-	scan = systable_beginscan(inhRel, InheritsRelidSeqnoIndexId, true,
-							  NULL, 2, key);
-	tuple = systable_getnext(scan);
-	if (HeapTupleIsValid(tuple))
-	{
-		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
-
-		result = form->inhparent;
-	}
-
-	systable_endscan(scan);
-
-	return result;
-}
-
-/*
  * get_partition_ancestors
  *		Obtain ancestors of given relation
  *
@@ -1471,24 +273,6 @@ get_partition_ancestors(Oid relid)
 }
 
 /*
- * get_partition_ancestors_worker
- *		recursive worker for get_partition_ancestors
- */
-static void
-get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
-{
-	Oid			parentOid;
-
-	/* Recursion ends at the topmost level, ie., when there's no parent */
-	parentOid = get_partition_parent_worker(inhRel, relid);
-	if (parentOid == InvalidOid)
-		return;
-
-	*ancestors = lappend_oid(*ancestors, parentOid);
-	get_partition_ancestors_worker(inhRel, parentOid, ancestors);
-}
-
-/*
  * get_qual_from_partbound
  *		Given a parser node for partition bound, return the list of executable
  *		expressions as partition constraint
@@ -1573,54 +357,375 @@ map_partition_varattnos(List *expr, int fromrel_varno,
 }
 
 /*
- * RelationGetPartitionQual
+ * get_default_partition_oid
  *
- * Returns a list of partition quals
+ * Given a relation OID, return the OID of the default partition, if one
+ * exists.  Use get_default_oid_from_partdesc where possible, for
+ * efficiency.
  */
-List *
-RelationGetPartitionQual(Relation rel)
+Oid
+get_default_partition_oid(Oid parentId)
 {
-	/* Quick exit */
-	if (!rel->rd_rel->relispartition)
-		return NIL;
+	HeapTuple	tuple;
+	Oid			defaultPartId = InvalidOid;
 
-	return generate_partition_qual(rel);
+	tuple = SearchSysCache1(PARTRELID, ObjectIdGetDatum(parentId));
+
+	if (HeapTupleIsValid(tuple))
+	{
+		Form_pg_partitioned_table part_table_form;
+
+		part_table_form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+		defaultPartId = part_table_form->partdefid;
+		ReleaseSysCache(tuple);
+	}
+
+	return defaultPartId;
 }
 
 /*
- * get_partition_qual_relid
+ * update_default_partition_oid
  *
- * Returns an expression tree describing the passed-in relation's partition
- * constraint. If there is no partition constraint returns NULL; this can
- * happen if the default partition is the only partition.
+ * Update pg_partition_table.partdefid with a new default partition OID.
  */
-Expr *
-get_partition_qual_relid(Oid relid)
+void
+update_default_partition_oid(Oid parentId, Oid defaultPartId)
 {
-	Relation	rel = heap_open(relid, AccessShareLock);
-	Expr	   *result = NULL;
-	List	   *and_args;
+	HeapTuple	tuple;
+	Relation	pg_partitioned_table;
+	Form_pg_partitioned_table part_table_form;
 
-	/* Do the work only if this relation is a partition. */
-	if (rel->rd_rel->relispartition)
+	pg_partitioned_table = heap_open(PartitionedRelationId, RowExclusiveLock);
+
+	tuple = SearchSysCacheCopy1(PARTRELID, ObjectIdGetDatum(parentId));
+
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for partition key of relation %u",
+			 parentId);
+
+	part_table_form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	part_table_form->partdefid = defaultPartId;
+	CatalogTupleUpdate(pg_partitioned_table, &tuple->t_self, tuple);
+
+	heap_freetuple(tuple);
+	heap_close(pg_partitioned_table, RowExclusiveLock);
+}
+
+/*
+ * get_proposed_default_constraint
+ *
+ * This function returns the negation of new_part_constraints, which
+ * would be an integral part of the default partition constraints after
+ * addition of the partition to which the new_part_constraints belongs.
+ */
+List *
+get_proposed_default_constraint(List *new_part_constraints)
+{
+	Expr	   *defPartConstraint;
+
+	defPartConstraint = make_ands_explicit(new_part_constraints);
+
+	/*
+	 * Derive the partition constraints of default partition by negating the
+	 * given partition constraints. The partition constraint never evaluates
+	 * to NULL, so negating it like this is safe.
+	 */
+	defPartConstraint = makeBoolExpr(NOT_EXPR,
+									 list_make1(defPartConstraint),
+									 -1);
+
+	/* Simplify, to put the negated expression into canonical form */
+	defPartConstraint =
+		(Expr *) eval_const_expressions(NULL,
+										(Node *) defPartConstraint);
+	defPartConstraint = canonicalize_qual(defPartConstraint, true);
+
+	return make_ands_implicit(defPartConstraint);
+}
+
+/*
+ * satisfies_hash_partition
+ *
+ * This is an SQL-callable function for use in hash partition constraints.
+ * The first three arguments are the parent table OID, modulus, and remainder.
+ * The remaining arguments are the value of the partitioning columns (or
+ * expressions); these are hashed and the results are combined into a single
+ * hash value by calling hash_combine64.
+ *
+ * Returns true if remainder produced when this computed single hash value is
+ * divided by the given modulus is equal to given remainder, otherwise false.
+ *
+ * See get_qual_for_hash() for usage.
+ */
+Datum
+satisfies_hash_partition(PG_FUNCTION_ARGS)
+{
+	typedef struct ColumnsHashData
 	{
-		and_args = generate_partition_qual(rel);
+		Oid			relid;
+		int			nkeys;
+		Oid			variadic_type;
+		int16		variadic_typlen;
+		bool		variadic_typbyval;
+		char		variadic_typalign;
+		FmgrInfo	partsupfunc[PARTITION_MAX_KEYS];
+	} ColumnsHashData;
+	Oid			parentId;
+	int			modulus;
+	int			remainder;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+	ColumnsHashData *my_extra;
+	uint64		rowHash = 0;
 
-		if (and_args == NIL)
-			result = NULL;
-		else if (list_length(and_args) > 1)
-			result = makeBoolExpr(AND_EXPR, and_args, -1);
+	/* Return null if the parent OID, modulus, or remainder is NULL. */
+	if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2))
+		PG_RETURN_NULL();
+	parentId = PG_GETARG_OID(0);
+	modulus = PG_GETARG_INT32(1);
+	remainder = PG_GETARG_INT32(2);
+
+	/* Sanity check modulus and remainder. */
+	if (modulus <= 0)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("modulus for hash partition must be a positive integer")));
+	if (remainder < 0)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("remainder for hash partition must be a non-negative integer")));
+	if (remainder >= modulus)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("remainder for hash partition must be less than modulus")));
+
+	/*
+	 * Cache hash function information.
+	 */
+	my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+	if (my_extra == NULL || my_extra->relid != parentId)
+	{
+		Relation	parent;
+		PartitionKey key;
+		int			j;
+
+		/* Open parent relation and fetch partition keyinfo */
+		parent = try_relation_open(parentId, AccessShareLock);
+		if (parent == NULL)
+			PG_RETURN_NULL();
+		key = RelationGetPartitionKey(parent);
+
+		/* Reject parent table that is not hash-partitioned. */
+		if (parent->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+			key->strategy != PARTITION_STRATEGY_HASH)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+					 errmsg("\"%s\" is not a hash partitioned table",
+							get_rel_name(parentId))));
+
+		if (!get_fn_expr_variadic(fcinfo->flinfo))
+		{
+			int			nargs = PG_NARGS() - 3;
+
+			/* complain if wrong number of column values */
+			if (key->partnatts != nargs)
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+						 errmsg("number of partitioning columns (%d) does not match number of partition keys provided (%d)",
+								key->partnatts, nargs)));
+
+			/* allocate space for our cache */
+			fcinfo->flinfo->fn_extra =
+				MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
+									   offsetof(ColumnsHashData, partsupfunc) +
+									   sizeof(FmgrInfo) * nargs);
+			my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+			my_extra->relid = parentId;
+			my_extra->nkeys = key->partnatts;
+
+			/* check argument types and save fmgr_infos */
+			for (j = 0; j < key->partnatts; ++j)
+			{
+				Oid			argtype = get_fn_expr_argtype(fcinfo->flinfo, j + 3);
+
+				if (argtype != key->parttypid[j] && !IsBinaryCoercible(argtype, key->parttypid[j]))
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+							 errmsg("column %d of the partition key has type \"%s\", but supplied value is of type \"%s\"",
+									j + 1, format_type_be(key->parttypid[j]), format_type_be(argtype))));
+
+				fmgr_info_copy(&my_extra->partsupfunc[j],
+							   &key->partsupfunc[j],
+							   fcinfo->flinfo->fn_mcxt);
+			}
+
+		}
 		else
-			result = linitial(and_args);
+		{
+			ArrayType  *variadic_array = PG_GETARG_ARRAYTYPE_P(3);
+
+			/* allocate space for our cache -- just one FmgrInfo in this case */
+			fcinfo->flinfo->fn_extra =
+				MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
+									   offsetof(ColumnsHashData, partsupfunc) +
+									   sizeof(FmgrInfo));
+			my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+			my_extra->relid = parentId;
+			my_extra->nkeys = key->partnatts;
+			my_extra->variadic_type = ARR_ELEMTYPE(variadic_array);
+			get_typlenbyvalalign(my_extra->variadic_type,
+								 &my_extra->variadic_typlen,
+								 &my_extra->variadic_typbyval,
+								 &my_extra->variadic_typalign);
+
+			/* check argument types */
+			for (j = 0; j < key->partnatts; ++j)
+				if (key->parttypid[j] != my_extra->variadic_type)
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+							 errmsg("column %d of the partition key has type \"%s\", but supplied value is of type \"%s\"",
+									j + 1,
+									format_type_be(key->parttypid[j]),
+									format_type_be(my_extra->variadic_type))));
+
+			fmgr_info_copy(&my_extra->partsupfunc[0],
+						   &key->partsupfunc[0],
+						   fcinfo->flinfo->fn_mcxt);
+		}
+
+		/* Hold lock until commit */
+		relation_close(parent, NoLock);
 	}
 
-	/* Keep the lock. */
-	heap_close(rel, NoLock);
+	if (!OidIsValid(my_extra->variadic_type))
+	{
+		int			nkeys = my_extra->nkeys;
+		int			i;
+
+		/*
+		 * For a non-variadic call, neither the number of arguments nor their
+		 * types can change across calls, so avoid the expense of rechecking
+		 * here.
+		 */
+
+		for (i = 0; i < nkeys; i++)
+		{
+			Datum		hash;
+
+			/* keys start from fourth argument of function. */
+			int			argno = i + 3;
+
+			if (PG_ARGISNULL(argno))
+				continue;
+
+			Assert(OidIsValid(my_extra->partsupfunc[i].fn_oid));
+
+			hash = FunctionCall2(&my_extra->partsupfunc[i],
+								 PG_GETARG_DATUM(argno),
+								 seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+	else
+	{
+		ArrayType  *variadic_array = PG_GETARG_ARRAYTYPE_P(3);
+		int			i;
+		int			nelems;
+		Datum	   *datum;
+		bool	   *isnull;
+
+		deconstruct_array(variadic_array,
+						  my_extra->variadic_type,
+						  my_extra->variadic_typlen,
+						  my_extra->variadic_typbyval,
+						  my_extra->variadic_typalign,
+						  &datum, &isnull, &nelems);
+
+		/* complain if wrong number of column values */
+		if (nelems != my_extra->nkeys)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+					 errmsg("number of partitioning columns (%d) does not match number of partition keys provided (%d)",
+							my_extra->nkeys, nelems)));
+
+		for (i = 0; i < nelems; i++)
+		{
+			Datum		hash;
+
+			if (isnull[i])
+				continue;
+
+			Assert(OidIsValid(my_extra->partsupfunc[0].fn_oid));
+
+			hash = FunctionCall2(&my_extra->partsupfunc[0],
+								 datum[i],
+								 seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
+
+/* Module-local functions */
+
+/*
+ * get_partition_parent_worker
+ *		Scan the pg_inherits relation to return the OID of the parent of the
+ *		given relation
+ */
+static Oid
+get_partition_parent_worker(Relation inhRel, Oid relid)
+{
+	SysScanDesc scan;
+	ScanKeyData key[2];
+	Oid			result = InvalidOid;
+	HeapTuple	tuple;
+
+	ScanKeyInit(&key[0],
+				Anum_pg_inherits_inhrelid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(relid));
+	ScanKeyInit(&key[1],
+				Anum_pg_inherits_inhseqno,
+				BTEqualStrategyNumber, F_INT4EQ,
+				Int32GetDatum(1));
+
+	scan = systable_beginscan(inhRel, InheritsRelidSeqnoIndexId, true,
+							  NULL, 2, key);
+	tuple = systable_getnext(scan);
+	if (HeapTupleIsValid(tuple))
+	{
+		Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
+
+		result = form->inhparent;
+	}
+
+	systable_endscan(scan);
 
 	return result;
 }
 
-/* Module-local functions */
+/*
+ * get_partition_ancestors_worker
+ *		recursive worker for get_partition_ancestors
+ */
+static void
+get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
+{
+	Oid			parentOid;
+
+	/* Recursion ends at the topmost level, ie., when there's no parent */
+	parentOid = get_partition_parent_worker(inhRel, relid);
+	if (parentOid == InvalidOid)
+		return;
+
+	*ancestors = lappend_oid(*ancestors, parentOid);
+	get_partition_ancestors_worker(inhRel, parentOid, ancestors);
+}
 
 /*
  * get_partition_operator
@@ -2531,1065 +1636,3 @@ get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
 
 	return result;
 }
-
-/*
- * generate_partition_qual
- *
- * Generate partition predicate from rel's partition bound expression. The
- * function returns a NIL list if there is no predicate.
- *
- * Result expression tree is stored CacheMemoryContext to ensure it survives
- * as long as the relcache entry. But we should be running in a less long-lived
- * working context. To avoid leaking cache memory if this routine fails partway
- * through, we build in working memory and then copy the completed structure
- * into cache memory.
- */
-static List *
-generate_partition_qual(Relation rel)
-{
-	HeapTuple	tuple;
-	MemoryContext oldcxt;
-	Datum		boundDatum;
-	bool		isnull;
-	PartitionBoundSpec *bound;
-	List	   *my_qual = NIL,
-			   *result = NIL;
-	Relation	parent;
-	bool		found_whole_row;
-
-	/* Guard against stack overflow due to overly deep partition tree */
-	check_stack_depth();
-
-	/* Quick copy */
-	if (rel->rd_partcheck != NIL)
-		return copyObject(rel->rd_partcheck);
-
-	/* Grab at least an AccessShareLock on the parent table */
-	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
-					   AccessShareLock);
-
-	/* Get pg_class.relpartbound */
-	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u",
-			 RelationGetRelid(rel));
-
-	boundDatum = SysCacheGetAttr(RELOID, tuple,
-								 Anum_pg_class_relpartbound,
-								 &isnull);
-	if (isnull)					/* should not happen */
-		elog(ERROR, "relation \"%s\" has relpartbound = null",
-			 RelationGetRelationName(rel));
-	bound = castNode(PartitionBoundSpec,
-					 stringToNode(TextDatumGetCString(boundDatum)));
-	ReleaseSysCache(tuple);
-
-	my_qual = get_qual_from_partbound(rel, parent, bound);
-
-	/* 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;
-
-	/*
-	 * Change Vars to have partition's attnos instead of the parent's. We do
-	 * this after we concatenate the parent's quals, because we want every Var
-	 * in it to bear this relation's attnos. It's safe to assume varno = 1
-	 * here.
-	 */
-	result = map_partition_varattnos(result, 1, rel, parent,
-									 &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");
-
-	/* Save a copy in the relcache */
-	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
-	rel->rd_partcheck = copyObject(result);
-	MemoryContextSwitchTo(oldcxt);
-
-	/* Keep the parent locked until commit */
-	heap_close(parent, NoLock);
-
-	return result;
-}
-
-/*
- * get_partition_for_tuple
- *		Finds partition of relation which accepts the partition key specified
- *		in values and isnull
- *
- * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
- * found or -1 if none found.
- */
-int
-get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
-{
-	int			bound_offset;
-	int			part_index = -1;
-	PartitionKey key = RelationGetPartitionKey(relation);
-	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
-
-	/* Route as appropriate based on partitioning strategy. */
-	switch (key->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-			{
-				PartitionBoundInfo boundinfo = partdesc->boundinfo;
-				int			greatest_modulus = get_greatest_modulus(boundinfo);
-				uint64		rowHash = compute_hash_value(key->partnatts,
-														 key->partsupfunc,
-														 values, isnull);
-
-				part_index = boundinfo->indexes[rowHash % greatest_modulus];
-			}
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			if (isnull[0])
-			{
-				if (partition_bound_accepts_nulls(partdesc->boundinfo))
-					part_index = partdesc->boundinfo->null_index;
-			}
-			else
-			{
-				bool		equal = false;
-
-				bound_offset = partition_list_bsearch(key->partsupfunc,
-													  key->partcollation,
-													  partdesc->boundinfo,
-													  values[0], &equal);
-				if (bound_offset >= 0 && equal)
-					part_index = partdesc->boundinfo->indexes[bound_offset];
-			}
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			{
-				bool		equal = false,
-							range_partkey_has_null = false;
-				int			i;
-
-				/*
-				 * No range includes NULL, so this will be accepted by the
-				 * default partition if there is one, and otherwise rejected.
-				 */
-				for (i = 0; i < key->partnatts; i++)
-				{
-					if (isnull[i])
-					{
-						range_partkey_has_null = true;
-						break;
-					}
-				}
-
-				if (!range_partkey_has_null)
-				{
-					bound_offset = partition_range_datum_bsearch(key->partsupfunc,
-																 key->partcollation,
-																 partdesc->boundinfo,
-																 key->partnatts,
-																 values,
-																 &equal);
-
-					/*
-					 * The bound at bound_offset is less than or equal to the
-					 * tuple value, so the bound at offset+1 is the upper
-					 * bound of the partition we're looking for, if there
-					 * actually exists one.
-					 */
-					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
-				}
-			}
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
-	}
-
-	/*
-	 * part_index < 0 means we failed to find a partition of this parent. Use
-	 * the default partition, if there is one.
-	 */
-	if (part_index < 0)
-		part_index = partdesc->boundinfo->default_index;
-
-	return part_index;
-}
-
-/*
- * Checks if any of the 'attnums' is a partition key attribute for rel
- *
- * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
- * partition key expression.  It's possible for a column to be both used
- * directly and as part of an expression; if that happens, *used_in_expr may
- * end up as either true or false.  That's OK for current uses of this
- * function, because *used_in_expr is only used to tailor the error message
- * text.
- */
-bool
-has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr)
-{
-	PartitionKey key;
-	int			partnatts;
-	List	   *partexprs;
-	ListCell   *partexprs_item;
-	int			i;
-
-	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-		return false;
-
-	key = RelationGetPartitionKey(rel);
-	partnatts = get_partition_natts(key);
-	partexprs = get_partition_exprs(key);
-
-	partexprs_item = list_head(partexprs);
-	for (i = 0; i < partnatts; i++)
-	{
-		AttrNumber	partattno = get_partition_col_attnum(key, i);
-
-		if (partattno != 0)
-		{
-			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
-							  attnums))
-			{
-				if (used_in_expr)
-					*used_in_expr = false;
-				return true;
-			}
-		}
-		else
-		{
-			/* Arbitrary expression */
-			Node	   *expr = (Node *) lfirst(partexprs_item);
-			Bitmapset  *expr_attrs = NULL;
-
-			/* Find all attributes referenced */
-			pull_varattnos(expr, 1, &expr_attrs);
-			partexprs_item = lnext(partexprs_item);
-
-			if (bms_overlap(attnums, expr_attrs))
-			{
-				if (used_in_expr)
-					*used_in_expr = true;
-				return true;
-			}
-		}
-	}
-
-	return false;
-}
-
-/*
- * qsort_partition_hbound_cmp
- *
- * We sort hash bounds by modulus, then by remainder.
- */
-static int32
-qsort_partition_hbound_cmp(const void *a, const void *b)
-{
-	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
-	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
-
-	return partition_hbound_cmp(h1->modulus, h1->remainder,
-								h2->modulus, h2->remainder);
-}
-
-/*
- * partition_hbound_cmp
- *
- * Compares modulus first, then remainder if modulus are equal.
- */
-static int32
-partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
-{
-	if (modulus1 < modulus2)
-		return -1;
-	if (modulus1 > modulus2)
-		return 1;
-	if (modulus1 == modulus2 && remainder1 != remainder2)
-		return (remainder1 > remainder2) ? 1 : -1;
-	return 0;
-}
-
-/*
- * qsort_partition_list_value_cmp
- *
- * Compare two list partition bound datums
- */
-static int32
-qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
-{
-	Datum		val1 = (*(const PartitionListValue **) a)->value,
-				val2 = (*(const PartitionListValue **) b)->value;
-	PartitionKey key = (PartitionKey) arg;
-
-	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-										   key->partcollation[0],
-										   val1, val2));
-}
-
-/*
- * make_one_range_bound
- *
- * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
- * and a flag telling whether the bound is lower or not.  Made into a function
- * because there are multiple sites that want to use this facility.
- */
-static PartitionRangeBound *
-make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
-{
-	PartitionRangeBound *bound;
-	ListCell   *lc;
-	int			i;
-
-	Assert(datums != NIL);
-
-	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
-	bound->index = index;
-	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
-	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
-													  sizeof(PartitionRangeDatumKind));
-	bound->lower = lower;
-
-	i = 0;
-	foreach(lc, datums)
-	{
-		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
-
-		/* What's contained in this range datum? */
-		bound->kind[i] = datum->kind;
-
-		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
-		{
-			Const	   *val = castNode(Const, datum->value);
-
-			if (val->constisnull)
-				elog(ERROR, "invalid range bound datum");
-			bound->datums[i] = val->constvalue;
-		}
-
-		i++;
-	}
-
-	return bound;
-}
-
-/* Used when sorting range bounds across all range partitions */
-static int32
-qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
-{
-	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
-	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
-	PartitionKey key = (PartitionKey) arg;
-
-	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
-								key->partcollation, b1->datums, b1->kind,
-								b1->lower, b2);
-}
-
-/*
- * partition_rbound_cmp
- *
- * Return for two range bounds whether the 1st one (specified in datums1,
- * kind1, and lower1) is <, =, or > the bound specified in *b2.
- *
- * partnatts, partsupfunc and partcollation give the number of attributes in the
- * bounds to be compared, comparison function to be used and the collations of
- * attributes, respectively.
- *
- * Note that if the values of the two range bounds compare equal, then we take
- * into account whether they are upper or lower bounds, and an upper bound is
- * considered to be smaller than a lower bound. This is important to the way
- * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
- * structure, which only stores the upper bound of a common boundary between
- * two contiguous partitions.
- */
-static int32
-partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2)
-{
-	int32		cmpval = 0;		/* placate compiler */
-	int			i;
-	Datum	   *datums2 = b2->datums;
-	PartitionRangeDatumKind *kind2 = b2->kind;
-	bool		lower2 = b2->lower;
-
-	for (i = 0; i < partnatts; i++)
-	{
-		/*
-		 * First, handle cases where the column is unbounded, which should not
-		 * invoke the comparison procedure, and should not consider any later
-		 * columns. Note that the PartitionRangeDatumKind enum elements
-		 * compare the same way as the values they represent.
-		 */
-		if (kind1[i] < kind2[i])
-			return -1;
-		else if (kind1[i] > kind2[i])
-			return 1;
-		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
-
-			/*
-			 * The column bounds are both MINVALUE or both MAXVALUE. No later
-			 * columns should be considered, but we still need to compare
-			 * whether they are upper or lower bounds.
-			 */
-			break;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
-												 partcollation[i],
-												 datums1[i],
-												 datums2[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	/*
-	 * If the comparison is anything other than equal, we're done. If they
-	 * compare equal though, we still have to consider whether the boundaries
-	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
-	 * two.
-	 */
-	if (cmpval == 0 && lower1 != lower2)
-		cmpval = lower1 ? 1 : -1;
-
-	return cmpval;
-}
-
-/*
- * partition_rbound_datum_cmp
- *
- * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
- * is <, =, or > partition key of tuple (tuple_datums)
- *
- * n_tuple_datums, partsupfunc and partcollation give number of attributes in
- * the bounds to be compared, comparison function to be used and the collations
- * of attributes resp.
- *
- */
-static int32
-partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-						   Datum *tuple_datums, int n_tuple_datums)
-{
-	int			i;
-	int32		cmpval = -1;
-
-	for (i = 0; i < n_tuple_datums; i++)
-	{
-		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
-			return -1;
-		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
-			return 1;
-
-		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
-												 partcollation[i],
-												 rb_datums[i],
-												 tuple_datums[i]));
-		if (cmpval != 0)
-			break;
-	}
-
-	return cmpval;
-}
-
-/*
- * partition_list_bsearch
- *		Returns the index of the greatest bound datum that is less than equal
- * 		to the given value or -1 if all of the bound datums are greater
- *
- * *is_equal is set to true if the bound datum at the returned index is equal
- * to the input value.
- */
-static int
-partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-					   PartitionBoundInfo boundinfo,
-					   Datum value, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
-												 partcollation[0],
-												 boundinfo->datums[mid][0],
-												 value));
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given range bound or -1 if all of the range bounds are
- *		greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input range bound
- */
-static int
-partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
-						Oid *partcollation,
-						PartitionBoundInfo boundinfo,
-						PartitionRangeBound *probe, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_cmp(partnatts, partsupfunc, partcollation,
-									  boundinfo->datums[mid],
-									  boundinfo->kind[mid],
-									  (boundinfo->indexes[mid] == -1),
-									  probe);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_range_bsearch
- *		Returns the index of the greatest range bound that is less than or
- *		equal to the given tuple or -1 if all of the range bounds are greater
- *
- * *is_equal is set to true if the range bound at the returned index is equal
- * to the input tuple.
- */
-static int
-partition_range_datum_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-							  PartitionBoundInfo boundinfo,
-							  int nvalues, Datum *values, bool *is_equal)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval;
-
-		mid = (lo + hi + 1) / 2;
-		cmpval = partition_rbound_datum_cmp(partsupfunc,
-											partcollation,
-											boundinfo->datums[mid],
-											boundinfo->kind[mid],
-											values,
-											nvalues);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-			*is_equal = (cmpval == 0);
-
-			if (*is_equal)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * partition_hash_bsearch
- *		Returns the index of the greatest (modulus, remainder) pair that is
- *		less than or equal to the given (modulus, remainder) pair or -1 if
- *		all of them are greater
- */
-static int
-partition_hash_bsearch(PartitionBoundInfo boundinfo,
-					   int modulus, int remainder)
-{
-	int			lo,
-				hi,
-				mid;
-
-	lo = -1;
-	hi = boundinfo->ndatums - 1;
-	while (lo < hi)
-	{
-		int32		cmpval,
-					bound_modulus,
-					bound_remainder;
-
-		mid = (lo + hi + 1) / 2;
-		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
-		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
-		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
-									  modulus, remainder);
-		if (cmpval <= 0)
-		{
-			lo = mid;
-
-			if (cmpval == 0)
-				break;
-		}
-		else
-			hi = mid - 1;
-	}
-
-	return lo;
-}
-
-/*
- * get_default_oid_from_partdesc
- *
- * Given a partition descriptor, return the OID of the default partition, if
- * one exists; else, return InvalidOid.
- */
-Oid
-get_default_oid_from_partdesc(PartitionDesc partdesc)
-{
-	if (partdesc && partdesc->boundinfo &&
-		partition_bound_has_default(partdesc->boundinfo))
-		return partdesc->oids[partdesc->boundinfo->default_index];
-
-	return InvalidOid;
-}
-
-/*
- * get_default_partition_oid
- *
- * Given a relation OID, return the OID of the default partition, if one
- * exists.  Use get_default_oid_from_partdesc where possible, for
- * efficiency.
- */
-Oid
-get_default_partition_oid(Oid parentId)
-{
-	HeapTuple	tuple;
-	Oid			defaultPartId = InvalidOid;
-
-	tuple = SearchSysCache1(PARTRELID, ObjectIdGetDatum(parentId));
-
-	if (HeapTupleIsValid(tuple))
-	{
-		Form_pg_partitioned_table part_table_form;
-
-		part_table_form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-		defaultPartId = part_table_form->partdefid;
-		ReleaseSysCache(tuple);
-	}
-
-	return defaultPartId;
-}
-
-/*
- * update_default_partition_oid
- *
- * Update pg_partition_table.partdefid with a new default partition OID.
- */
-void
-update_default_partition_oid(Oid parentId, Oid defaultPartId)
-{
-	HeapTuple	tuple;
-	Relation	pg_partitioned_table;
-	Form_pg_partitioned_table part_table_form;
-
-	pg_partitioned_table = heap_open(PartitionedRelationId, RowExclusiveLock);
-
-	tuple = SearchSysCacheCopy1(PARTRELID, ObjectIdGetDatum(parentId));
-
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for partition key of relation %u",
-			 parentId);
-
-	part_table_form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	part_table_form->partdefid = defaultPartId;
-	CatalogTupleUpdate(pg_partitioned_table, &tuple->t_self, tuple);
-
-	heap_freetuple(tuple);
-	heap_close(pg_partitioned_table, RowExclusiveLock);
-}
-
-/*
- * get_proposed_default_constraint
- *
- * This function returns the negation of new_part_constraints, which
- * would be an integral part of the default partition constraints after
- * addition of the partition to which the new_part_constraints belongs.
- */
-List *
-get_proposed_default_constraint(List *new_part_constraints)
-{
-	Expr	   *defPartConstraint;
-
-	defPartConstraint = make_ands_explicit(new_part_constraints);
-
-	/*
-	 * Derive the partition constraints of default partition by negating the
-	 * given partition constraints. The partition constraint never evaluates
-	 * to NULL, so negating it like this is safe.
-	 */
-	defPartConstraint = makeBoolExpr(NOT_EXPR,
-									 list_make1(defPartConstraint),
-									 -1);
-
-	/* Simplify, to put the negated expression into canonical form */
-	defPartConstraint =
-		(Expr *) eval_const_expressions(NULL,
-										(Node *) defPartConstraint);
-	defPartConstraint = canonicalize_qual(defPartConstraint, true);
-
-	return make_ands_implicit(defPartConstraint);
-}
-
-/*
- * get_partition_bound_num_indexes
- *
- * Returns the number of the entries in the partition bound indexes array.
- */
-static int
-get_partition_bound_num_indexes(PartitionBoundInfo bound)
-{
-	int			num_indexes;
-
-	Assert(bound);
-
-	switch (bound->strategy)
-	{
-		case PARTITION_STRATEGY_HASH:
-
-			/*
-			 * The number of the entries in the indexes array is same as the
-			 * greatest modulus.
-			 */
-			num_indexes = get_greatest_modulus(bound);
-			break;
-
-		case PARTITION_STRATEGY_LIST:
-			num_indexes = bound->ndatums;
-			break;
-
-		case PARTITION_STRATEGY_RANGE:
-			/* Range partitioned table has an extra index. */
-			num_indexes = bound->ndatums + 1;
-			break;
-
-		default:
-			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) bound->strategy);
-	}
-
-	return num_indexes;
-}
-
-/*
- * get_greatest_modulus
- *
- * Returns the greatest modulus of the hash partition bound. The greatest
- * modulus will be at the end of the datums array because hash partitions are
- * arranged in the ascending order of their modulus and remainders.
- */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
-{
-	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
-	Assert(bound->datums && bound->ndatums > 0);
-	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
-
-	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
-}
-
-/*
- * compute_hash_value
- *
- * Compute the hash value for given not null partition key values.
- */
-static uint64
-compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
-				   Datum *values, bool *isnull)
-{
-	int			i;
-	uint64		rowHash = 0;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-
-	for (i = 0; i < partnatts; i++)
-	{
-		if (!isnull[i])
-		{
-			Datum		hash;
-
-			Assert(OidIsValid(partsupfunc[i].fn_oid));
-
-			/*
-			 * Compute hash for each datum value by calling respective
-			 * datatype-specific hash functions of each partition key
-			 * attribute.
-			 */
-			hash = FunctionCall2(&partsupfunc[i], values[i], seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	return rowHash;
-}
-
-/*
- * satisfies_hash_partition
- *
- * This is an SQL-callable function for use in hash partition constraints.
- * The first three arguments are the parent table OID, modulus, and remainder.
- * The remaining arguments are the value of the partitioning columns (or
- * expressions); these are hashed and the results are combined into a single
- * hash value by calling hash_combine64.
- *
- * Returns true if remainder produced when this computed single hash value is
- * divided by the given modulus is equal to given remainder, otherwise false.
- *
- * See get_qual_for_hash() for usage.
- */
-Datum
-satisfies_hash_partition(PG_FUNCTION_ARGS)
-{
-	typedef struct ColumnsHashData
-	{
-		Oid			relid;
-		int			nkeys;
-		Oid			variadic_type;
-		int16		variadic_typlen;
-		bool		variadic_typbyval;
-		char		variadic_typalign;
-		FmgrInfo	partsupfunc[PARTITION_MAX_KEYS];
-	} ColumnsHashData;
-	Oid			parentId;
-	int			modulus;
-	int			remainder;
-	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
-	ColumnsHashData *my_extra;
-	uint64		rowHash = 0;
-
-	/* Return null if the parent OID, modulus, or remainder is NULL. */
-	if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2))
-		PG_RETURN_NULL();
-	parentId = PG_GETARG_OID(0);
-	modulus = PG_GETARG_INT32(1);
-	remainder = PG_GETARG_INT32(2);
-
-	/* Sanity check modulus and remainder. */
-	if (modulus <= 0)
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-				 errmsg("modulus for hash partition must be a positive integer")));
-	if (remainder < 0)
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-				 errmsg("remainder for hash partition must be a non-negative integer")));
-	if (remainder >= modulus)
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-				 errmsg("remainder for hash partition must be less than modulus")));
-
-	/*
-	 * Cache hash function information.
-	 */
-	my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
-	if (my_extra == NULL || my_extra->relid != parentId)
-	{
-		Relation	parent;
-		PartitionKey key;
-		int			j;
-
-		/* Open parent relation and fetch partition keyinfo */
-		parent = try_relation_open(parentId, AccessShareLock);
-		if (parent == NULL)
-			PG_RETURN_NULL();
-		key = RelationGetPartitionKey(parent);
-
-		/* Reject parent table that is not hash-partitioned. */
-		if (parent->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
-			key->strategy != PARTITION_STRATEGY_HASH)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-					 errmsg("\"%s\" is not a hash partitioned table",
-							get_rel_name(parentId))));
-
-		if (!get_fn_expr_variadic(fcinfo->flinfo))
-		{
-			int			nargs = PG_NARGS() - 3;
-
-			/* complain if wrong number of column values */
-			if (key->partnatts != nargs)
-				ereport(ERROR,
-						(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-						 errmsg("number of partitioning columns (%d) does not match number of partition keys provided (%d)",
-								key->partnatts, nargs)));
-
-			/* allocate space for our cache */
-			fcinfo->flinfo->fn_extra =
-				MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
-									   offsetof(ColumnsHashData, partsupfunc) +
-									   sizeof(FmgrInfo) * nargs);
-			my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
-			my_extra->relid = parentId;
-			my_extra->nkeys = key->partnatts;
-
-			/* check argument types and save fmgr_infos */
-			for (j = 0; j < key->partnatts; ++j)
-			{
-				Oid			argtype = get_fn_expr_argtype(fcinfo->flinfo, j + 3);
-
-				if (argtype != key->parttypid[j] && !IsBinaryCoercible(argtype, key->parttypid[j]))
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-							 errmsg("column %d of the partition key has type \"%s\", but supplied value is of type \"%s\"",
-									j + 1, format_type_be(key->parttypid[j]), format_type_be(argtype))));
-
-				fmgr_info_copy(&my_extra->partsupfunc[j],
-							   &key->partsupfunc[j],
-							   fcinfo->flinfo->fn_mcxt);
-			}
-
-		}
-		else
-		{
-			ArrayType  *variadic_array = PG_GETARG_ARRAYTYPE_P(3);
-
-			/* allocate space for our cache -- just one FmgrInfo in this case */
-			fcinfo->flinfo->fn_extra =
-				MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
-									   offsetof(ColumnsHashData, partsupfunc) +
-									   sizeof(FmgrInfo));
-			my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
-			my_extra->relid = parentId;
-			my_extra->nkeys = key->partnatts;
-			my_extra->variadic_type = ARR_ELEMTYPE(variadic_array);
-			get_typlenbyvalalign(my_extra->variadic_type,
-								 &my_extra->variadic_typlen,
-								 &my_extra->variadic_typbyval,
-								 &my_extra->variadic_typalign);
-
-			/* check argument types */
-			for (j = 0; j < key->partnatts; ++j)
-				if (key->parttypid[j] != my_extra->variadic_type)
-					ereport(ERROR,
-							(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-							 errmsg("column %d of the partition key has type \"%s\", but supplied value is of type \"%s\"",
-									j + 1,
-									format_type_be(key->parttypid[j]),
-									format_type_be(my_extra->variadic_type))));
-
-			fmgr_info_copy(&my_extra->partsupfunc[0],
-						   &key->partsupfunc[0],
-						   fcinfo->flinfo->fn_mcxt);
-		}
-
-		/* Hold lock until commit */
-		relation_close(parent, NoLock);
-	}
-
-	if (!OidIsValid(my_extra->variadic_type))
-	{
-		int			nkeys = my_extra->nkeys;
-		int			i;
-
-		/*
-		 * For a non-variadic call, neither the number of arguments nor their
-		 * types can change across calls, so avoid the expense of rechecking
-		 * here.
-		 */
-
-		for (i = 0; i < nkeys; i++)
-		{
-			Datum		hash;
-
-			/* keys start from fourth argument of function. */
-			int			argno = i + 3;
-
-			if (PG_ARGISNULL(argno))
-				continue;
-
-			Assert(OidIsValid(my_extra->partsupfunc[i].fn_oid));
-
-			hash = FunctionCall2(&my_extra->partsupfunc[i],
-								 PG_GETARG_DATUM(argno),
-								 seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-	else
-	{
-		ArrayType  *variadic_array = PG_GETARG_ARRAYTYPE_P(3);
-		int			i;
-		int			nelems;
-		Datum	   *datum;
-		bool	   *isnull;
-
-		deconstruct_array(variadic_array,
-						  my_extra->variadic_type,
-						  my_extra->variadic_typlen,
-						  my_extra->variadic_typbyval,
-						  my_extra->variadic_typalign,
-						  &datum, &isnull, &nelems);
-
-		/* complain if wrong number of column values */
-		if (nelems != my_extra->nkeys)
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-					 errmsg("number of partitioning columns (%d) does not match number of partition keys provided (%d)",
-							my_extra->nkeys, nelems)));
-
-		for (i = 0; i < nelems; i++)
-		{
-			Datum		hash;
-
-			if (isnull[i])
-				continue;
-
-			Assert(OidIsValid(my_extra->partsupfunc[0].fn_oid));
-
-			hash = FunctionCall2(&my_extra->partsupfunc[0],
-								 datum[i],
-								 seed);
-
-			/* Form a single 64-bit hash value */
-			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
-		}
-	}
-
-	PG_RETURN_BOOL(rowHash % modulus == remainder);
-}
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 9a107aba56..6347ed6191 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -42,7 +42,6 @@
 #include "access/transam.h"
 #include "access/xact.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_publication.h"
 #include "commands/matview.h"
 #include "commands/trigger.h"
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 9a13188649..614cd1d372 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -22,6 +22,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..f86b6df2ca 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -15,13 +15,13 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/prep.h"
 #include "utils/lsyscache.h"
+#include "utils/partcache.h"
 #include "utils/memutils.h"
 
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 6e510f9d94..bc2fd520e1 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -33,7 +33,6 @@
 #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"
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 0231f8bf7c..01ed29fa98 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,7 +27,6 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_statistic_ext.h"
 #include "foreign/fdwapi.h"
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index da8f0f93fc..b471b39667 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -17,7 +17,6 @@
 #include <limits.h>
 
 #include "miscadmin.h"
-#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
@@ -28,6 +27,7 @@
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
+#include "utils/partcache.h"
 
 
 typedef struct JoinHashEntry
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index f8fc7f83f9..bbd2287df8 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -24,7 +24,6 @@
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
-#include "catalog/partition.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_authid.h"
diff --git a/src/backend/utils/cache/Makefile b/src/backend/utils/cache/Makefile
index a943f8ea4b..94511eaf54 100644
--- a/src/backend/utils/cache/Makefile
+++ b/src/backend/utils/cache/Makefile
@@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
-	relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
-	typcache.o ts_cache.o
+OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o partcache.o \
+    relcache.o relmapper.o relfilenodemap.o spccache.o syscache.o \
+    lsyscache.o typcache.o ts_cache.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/cache/partcache.c b/src/backend/utils/cache/partcache.c
new file mode 100644
index 0000000000..8de6a167d0
--- /dev/null
+++ b/src/backend/utils/cache/partcache.c
@@ -0,0 +1,2151 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.c
+ *		Partitioning related cache data structures and manipulation functions
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *		  src/backend/utils/cache/partcache.c
+ *
+ *-------------------------------------------------------------------------
+*/
+
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "access/nbtree.h"
+#include "access/sysattr.h"
+#include "catalog/partition.h"
+#include "catalog/pg_inherits_fn.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
+#include "utils/builtins.h"
+#include "utils/datum.h"
+#include "utils/hashutils.h"
+#include "utils/lsyscache.h"
+#include "utils/memutils.h"
+#include "utils/ruleutils.h"
+#include "utils/syscache.h"
+
+/*
+ * Partition bounds are represented using one of the following structs when
+ * they are first read into the backend memory from the catalog (wherein they
+ * are stored in in their parser represenation).  Especially, the bound
+ * comparison and sort functions expect the bounds passed to them to be of
+ * this form.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int			modulus;
+	int			remainder;
+	int			index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+	int			index;
+	Datum		value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+	int			index;
+	Datum	   *datums;			/* range bound datums */
+	PartitionRangeDatumKind *kind;	/* the kind of each datum */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+static List *generate_partition_qual(Relation rel);
+
+static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
+					 int remainder2);
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
+static int partition_hash_bsearch(PartitionBoundInfo boundinfo,
+					   int modulus, int remainder);
+
+static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
+							void *arg);
+static int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal);
+
+static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
+					 List *datums, bool lower);
+static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					 Oid *partcollation, Datum *datums1,
+					 PartitionRangeDatumKind *kind1, bool lower1,
+					 PartitionRangeBound *b2);
+static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
+					void *arg);
+static int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
+						   Oid *partcollation,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums);
+static int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
+						Oid *partcollation,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal);
+static int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
+							  Oid *partcollation,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal);
+
+static int	get_partition_bound_num_indexes(PartitionBoundInfo b);
+
+/*
+ * RelationBuildPartitionKey
+ *		Build and attach to relcache partition key data of relation
+ *
+ * Partitioning key data is a complex structure; to avoid complicated logic to
+ * free individual elements whenever the relcache entry is flushed, we give it
+ * its own memory context, child of CacheMemoryContext, which can easily be
+ * deleted on its own.  To avoid leaking memory in that context in case of an
+ * error partway through this function, the context is initially created as a
+ * child of CurTransactionContext and only re-parented to CacheMemoryContext
+ * at the end, when no further errors are possible.  Also, we don't make this
+ * context the current context except in very brief code sections, out of fear
+ * that some of our callees allocate memory on their own which would be leaked
+ * permanently.
+ */
+void
+RelationBuildPartitionKey(Relation relation)
+{
+	Form_pg_partitioned_table form;
+	HeapTuple	tuple;
+	bool		isnull;
+	int			i;
+	PartitionKey key;
+	AttrNumber *attrs;
+	oidvector  *opclass;
+	oidvector  *collation;
+	ListCell   *partexprs_item;
+	Datum		datum;
+	MemoryContext partkeycxt,
+				oldcxt;
+	int16		procnum;
+
+	tuple = SearchSysCache1(PARTRELID,
+							ObjectIdGetDatum(RelationGetRelid(relation)));
+
+	/*
+	 * The following happens when we have created our pg_class entry but not
+	 * the pg_partitioned_table entry yet.
+	 */
+	if (!HeapTupleIsValid(tuple))
+		return;
+
+	partkeycxt = AllocSetContextCreate(CurTransactionContext,
+									   "partition key",
+									   ALLOCSET_SMALL_SIZES);
+	MemoryContextCopySetIdentifier(partkeycxt,
+								   RelationGetRelationName(relation));
+
+	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
+												sizeof(PartitionKeyData));
+
+	/* Fixed-length attributes */
+	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+	key->strategy = form->partstrat;
+	key->partnatts = form->partnatts;
+
+	/*
+	 * We can rely on the first variable-length attribute being mapped to the
+	 * relevant field of the catalog's C struct, because all previous
+	 * attributes are non-nullable and fixed-length.
+	 */
+	attrs = form->partattrs.values;
+
+	/* But use the hard way to retrieve further variable-length attributes */
+	/* Operator class */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partclass, &isnull);
+	Assert(!isnull);
+	opclass = (oidvector *) DatumGetPointer(datum);
+
+	/* Collation */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partcollation, &isnull);
+	Assert(!isnull);
+	collation = (oidvector *) DatumGetPointer(datum);
+
+	/* Expressions */
+	datum = SysCacheGetAttr(PARTRELID, tuple,
+							Anum_pg_partitioned_table_partexprs, &isnull);
+	if (!isnull)
+	{
+		char	   *exprString;
+		Node	   *expr;
+
+		exprString = TextDatumGetCString(datum);
+		expr = stringToNode(exprString);
+		pfree(exprString);
+
+		/*
+		 * Run the expressions through const-simplification since the planner
+		 * will be comparing them to similarly-processed qual clause operands,
+		 * and may fail to detect valid matches without this step; fix
+		 * opfuncids while at it.  We don't need to bother with
+		 * canonicalize_qual() though, because partition expressions are not
+		 * full-fledged qualification clauses.
+		 */
+		expr = eval_const_expressions(NULL, expr);
+		fix_opfuncids(expr);
+
+		oldcxt = MemoryContextSwitchTo(partkeycxt);
+		key->partexprs = (List *) copyObject(expr);
+		MemoryContextSwitchTo(oldcxt);
+	}
+
+	oldcxt = MemoryContextSwitchTo(partkeycxt);
+	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
+	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
+
+	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+
+	/* Gather type and collation info as well */
+	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
+	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
+	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
+	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
+	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
+	MemoryContextSwitchTo(oldcxt);
+
+	/* determine support function number to search for */
+	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
+		HASHEXTENDED_PROC : BTORDER_PROC;
+
+	/* Copy partattrs and fill other per-attribute info */
+	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
+	partexprs_item = list_head(key->partexprs);
+	for (i = 0; i < key->partnatts; i++)
+	{
+		AttrNumber	attno = key->partattrs[i];
+		HeapTuple	opclasstup;
+		Form_pg_opclass opclassform;
+		Oid			funcid;
+
+		/* Collect opfamily information */
+		opclasstup = SearchSysCache1(CLAOID,
+									 ObjectIdGetDatum(opclass->values[i]));
+		if (!HeapTupleIsValid(opclasstup))
+			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
+
+		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
+		key->partopfamily[i] = opclassform->opcfamily;
+		key->partopcintype[i] = opclassform->opcintype;
+
+		/* Get a support function for the specified opfamily and datatypes */
+		funcid = get_opfamily_proc(opclassform->opcfamily,
+								   opclassform->opcintype,
+								   opclassform->opcintype,
+								   procnum);
+		if (!OidIsValid(funcid))
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
+							NameStr(opclassform->opcname),
+							(key->strategy == PARTITION_STRATEGY_HASH) ?
+							"hash" : "btree",
+							procnum,
+							format_type_be(opclassform->opcintype))));
+
+		fmgr_info(funcid, &key->partsupfunc[i]);
+
+		/* Collation */
+		key->partcollation[i] = collation->values[i];
+
+		/* Collect type information */
+		if (attno != 0)
+		{
+			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
+
+			key->parttypid[i] = att->atttypid;
+			key->parttypmod[i] = att->atttypmod;
+			key->parttypcoll[i] = att->attcollation;
+		}
+		else
+		{
+			if (partexprs_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			key->parttypid[i] = exprType(lfirst(partexprs_item));
+			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
+			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
+
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(key->parttypid[i],
+							 &key->parttyplen[i],
+							 &key->parttypbyval[i],
+							 &key->parttypalign[i]);
+
+		ReleaseSysCache(opclasstup);
+	}
+
+	ReleaseSysCache(tuple);
+
+	/*
+	 * Success --- reparent our context and make the relcache point to the
+	 * newly constructed key
+	 */
+	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
+	relation->rd_partkeycxt = partkeycxt;
+	relation->rd_partkey = key;
+}
+
+/*
+ * RelationBuildPartitionDesc
+ *		Form rel's partition descriptor
+ *
+ * Not flushed from the cache by RelationClearRelation() unless changed because
+ * of addition or removal of partition.
+ */
+void
+RelationBuildPartitionDesc(Relation rel)
+{
+	List	   *inhoids,
+			   *partoids;
+	Oid		   *oids = NULL;
+	List	   *boundspecs = NIL;
+	ListCell   *cell;
+	int			i,
+				nparts;
+	PartitionKey key = RelationGetPartitionKey(rel);
+	PartitionDesc result;
+	MemoryContext oldcxt;
+
+	int			ndatums = 0;
+	int			default_index = -1;
+
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
+	/* List partitioning specific */
+	PartitionListValue **all_values = NULL;
+	int			null_index = -1;
+
+	/* Range partitioning specific */
+	PartitionRangeBound **rbounds = NULL;
+
+	/*
+	 * The following could happen in situations where rel has a pg_class entry
+	 * but not the pg_partitioned_table entry yet.
+	 */
+	if (key == NULL)
+		return;
+
+	/* Get partition oids from pg_inherits */
+	inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
+
+	/* Collect bound spec nodes in a list */
+	i = 0;
+	partoids = NIL;
+	foreach(cell, inhoids)
+	{
+		Oid			inhrelid = lfirst_oid(cell);
+		HeapTuple	tuple;
+		Datum		datum;
+		bool		isnull;
+		Node	   *boundspec;
+
+		tuple = SearchSysCache1(RELOID, inhrelid);
+		if (!HeapTupleIsValid(tuple))
+			elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+		/*
+		 * It is possible that the pg_class tuple of a partition has not been
+		 * updated yet to set its relpartbound field.  The only case where
+		 * this happens is when we open the parent relation to check using its
+		 * partition descriptor that a new partition's bound does not overlap
+		 * some existing partition.
+		 */
+		if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
+		{
+			ReleaseSysCache(tuple);
+			continue;
+		}
+
+		datum = SysCacheGetAttr(RELOID, tuple,
+								Anum_pg_class_relpartbound,
+								&isnull);
+		Assert(!isnull);
+		boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+		/*
+		 * Sanity check: If the PartitionBoundSpec says this is the default
+		 * partition, its OID should correspond to whatever's stored in
+		 * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+		 */
+		if (castNode(PartitionBoundSpec, boundspec)->is_default)
+		{
+			Oid			partdefid;
+
+			partdefid = get_default_partition_oid(RelationGetRelid(rel));
+			if (partdefid != inhrelid)
+				elog(ERROR, "expected partdefid %u, but got %u",
+					 inhrelid, partdefid);
+		}
+
+		boundspecs = lappend(boundspecs, boundspec);
+		partoids = lappend_oid(partoids, inhrelid);
+		ReleaseSysCache(tuple);
+	}
+
+	nparts = list_length(partoids);
+
+	if (nparts > 0)
+	{
+		oids = (Oid *) palloc(nparts * sizeof(Oid));
+		i = 0;
+		foreach(cell, partoids)
+			oids[i++] = lfirst_oid(cell);
+
+		/* Convert from node to the internal representation */
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **)
+				palloc(nparts * sizeof(PartitionHashBound *));
+
+			i = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
+		{
+			List	   *non_null_values = NIL;
+
+			/*
+			 * Create a unified list of non-null values across all partitions.
+			 */
+			i = 0;
+			null_index = -1;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				ListCell   *c;
+
+				if (spec->strategy != PARTITION_STRATEGY_LIST)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
+				foreach(c, spec->listdatums)
+				{
+					Const	   *val = castNode(Const, lfirst(c));
+					PartitionListValue *list_value = NULL;
+
+					if (!val->constisnull)
+					{
+						list_value = (PartitionListValue *)
+							palloc0(sizeof(PartitionListValue));
+						list_value->index = i;
+						list_value->value = val->constvalue;
+					}
+					else
+					{
+						/*
+						 * Never put a null into the values array, flag
+						 * instead for the code further down below where we
+						 * construct the actual relcache struct.
+						 */
+						if (null_index != -1)
+							elog(ERROR, "found null more than once");
+						null_index = i;
+					}
+
+					if (list_value)
+						non_null_values = lappend(non_null_values,
+												  list_value);
+				}
+
+				i++;
+			}
+
+			ndatums = list_length(non_null_values);
+
+			/*
+			 * Collect all list values in one array. Alongside the value, we
+			 * also save the index of partition the value comes from.
+			 */
+			all_values = (PartitionListValue **) palloc(ndatums *
+														sizeof(PartitionListValue *));
+			i = 0;
+			foreach(cell, non_null_values)
+			{
+				PartitionListValue *src = lfirst(cell);
+
+				all_values[i] = (PartitionListValue *)
+					palloc(sizeof(PartitionListValue));
+				all_values[i]->value = src->value;
+				all_values[i]->index = src->index;
+				i++;
+			}
+
+			qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
+					  qsort_partition_list_value_cmp, (void *) key);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_RANGE)
+		{
+			int			k;
+			PartitionRangeBound **all_bounds,
+					   *prev;
+
+			all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
+														  sizeof(PartitionRangeBound *));
+
+			/*
+			 * Create a unified list of range bounds across all the
+			 * partitions.
+			 */
+			i = ndatums = 0;
+			foreach(cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
+													lfirst(cell));
+				PartitionRangeBound *lower,
+						   *upper;
+
+				if (spec->strategy != PARTITION_STRATEGY_RANGE)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the allbounds array
+				 * for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i++;
+					continue;
+				}
+
+				lower = make_one_range_bound(key, i, spec->lowerdatums,
+											 true);
+				upper = make_one_range_bound(key, i, spec->upperdatums,
+											 false);
+				all_bounds[ndatums++] = lower;
+				all_bounds[ndatums++] = upper;
+				i++;
+			}
+
+			Assert(ndatums == nparts * 2 ||
+				   (default_index != -1 && ndatums == (nparts - 1) * 2));
+
+			/* Sort all the bounds in ascending order */
+			qsort_arg(all_bounds, ndatums,
+					  sizeof(PartitionRangeBound *),
+					  qsort_partition_rbound_cmp,
+					  (void *) key);
+
+			/* Save distinct bounds from all_bounds into rbounds. */
+			rbounds = (PartitionRangeBound **)
+				palloc(ndatums * sizeof(PartitionRangeBound *));
+			k = 0;
+			prev = NULL;
+			for (i = 0; i < ndatums; i++)
+			{
+				PartitionRangeBound *cur = all_bounds[i];
+				bool		is_distinct = false;
+				int			j;
+
+				/* Is the current bound distinct from the previous one? */
+				for (j = 0; j < key->partnatts; j++)
+				{
+					Datum		cmpval;
+
+					if (prev == NULL || cur->kind[j] != prev->kind[j])
+					{
+						is_distinct = true;
+						break;
+					}
+
+					/*
+					 * If the bounds are both MINVALUE or MAXVALUE, stop now
+					 * and treat them as equal, since any values after this
+					 * point must be ignored.
+					 */
+					if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
+						break;
+
+					cmpval = FunctionCall2Coll(&key->partsupfunc[j],
+											   key->partcollation[j],
+											   cur->datums[j],
+											   prev->datums[j]);
+					if (DatumGetInt32(cmpval) != 0)
+					{
+						is_distinct = true;
+						break;
+					}
+				}
+
+				/*
+				 * Only if the bound is distinct save it into a temporary
+				 * array i.e. rbounds which is later copied into boundinfo
+				 * datums array.
+				 */
+				if (is_distinct)
+					rbounds[k++] = all_bounds[i];
+
+				prev = cur;
+			}
+
+			/* Update ndatums to hold the count of distinct datums. */
+			ndatums = k;
+		}
+		else
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/* Now build the actual relcache partition descriptor */
+	rel->rd_pdcxt = AllocSetContextCreate(CacheMemoryContext,
+										  "partition descriptor",
+										  ALLOCSET_DEFAULT_SIZES);
+	MemoryContextCopySetIdentifier(rel->rd_pdcxt,
+								   RelationGetRelationName(rel));
+
+	oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
+
+	result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
+	result->nparts = nparts;
+	if (nparts > 0)
+	{
+		PartitionBoundInfo boundinfo;
+		int		   *mapping;
+		int			next_index = 0;
+
+		result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
+
+		boundinfo = (PartitionBoundInfoData *)
+			palloc0(sizeof(PartitionBoundInfoData));
+		boundinfo->strategy = key->strategy;
+		boundinfo->default_index = -1;
+		boundinfo->ndatums = ndatums;
+		boundinfo->null_index = -1;
+		boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
+
+		/* Initialize mapping array with invalid values */
+		mapping = (int *) palloc(sizeof(int) * nparts);
+		for (i = 0; i < nparts; i++)
+			mapping[i] = -1;
+
+		switch (key->strategy)
+		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int			greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+
+					for (i = 0; i < greatest_modulus; i++)
+						boundinfo->indexes[i] = -1;
+
+					for (i = 0; i < nparts; i++)
+					{
+						int			modulus = hbounds[i]->modulus;
+						int			remainder = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(modulus);
+						boundinfo->datums[i][1] = Int32GetDatum(remainder);
+
+						while (remainder < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[remainder] == -1);
+							boundinfo->indexes[remainder] = i;
+							remainder += modulus;
+						}
+
+						mapping[hbounds[i]->index] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
+			case PARTITION_STRATEGY_LIST:
+				{
+					boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
+
+					/*
+					 * Copy values.  Indexes of individual values are mapped
+					 * to canonical values so that they match for any two list
+					 * partitioned tables with same number of partitions and
+					 * same lists per partition.  One way to canonicalize is
+					 * to assign the index in all_values[] of the smallest
+					 * value of each partition, as the index of all of the
+					 * partition's values.
+					 */
+					for (i = 0; i < ndatums; i++)
+					{
+						boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
+						boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
+															key->parttypbyval[0],
+															key->parttyplen[0]);
+
+						/* If the old index has no mapping, assign one */
+						if (mapping[all_values[i]->index] == -1)
+							mapping[all_values[i]->index] = next_index++;
+
+						boundinfo->indexes[i] = mapping[all_values[i]->index];
+					}
+
+					/*
+					 * If null-accepting partition has no mapped index yet,
+					 * assign one.  This could happen if such partition
+					 * accepts only null and hence not covered in the above
+					 * loop which only handled non-null values.
+					 */
+					if (null_index != -1)
+					{
+						Assert(null_index >= 0);
+						if (mapping[null_index] == -1)
+							mapping[null_index] = next_index++;
+						boundinfo->null_index = mapping[null_index];
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any value not
+						 * specified in the lists of other partitions, hence
+						 * it should not get mapped index while assigning
+						 * those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+
+					/* All partition must now have a valid mapping */
+					Assert(next_index == nparts);
+					break;
+				}
+
+			case PARTITION_STRATEGY_RANGE:
+				{
+					boundinfo->kind = (PartitionRangeDatumKind **)
+						palloc(ndatums *
+							   sizeof(PartitionRangeDatumKind *));
+					boundinfo->indexes = (int *) palloc((ndatums + 1) *
+														sizeof(int));
+
+					for (i = 0; i < ndatums; i++)
+					{
+						int			j;
+
+						boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
+																sizeof(Datum));
+						boundinfo->kind[i] = (PartitionRangeDatumKind *)
+							palloc(key->partnatts *
+								   sizeof(PartitionRangeDatumKind));
+						for (j = 0; j < key->partnatts; j++)
+						{
+							if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
+								boundinfo->datums[i][j] =
+									datumCopy(rbounds[i]->datums[j],
+											  key->parttypbyval[j],
+											  key->parttyplen[j]);
+							boundinfo->kind[i][j] = rbounds[i]->kind[j];
+						}
+
+						/*
+						 * There is no mapping for invalid indexes.
+						 *
+						 * Any lower bounds in the rbounds array have invalid
+						 * indexes assigned, because the values between the
+						 * previous bound (if there is one) and this (lower)
+						 * bound are not part of the range of any existing
+						 * partition.
+						 */
+						if (rbounds[i]->lower)
+							boundinfo->indexes[i] = -1;
+						else
+						{
+							int			orig_index = rbounds[i]->index;
+
+							/* If the old index has no mapping, assign one */
+							if (mapping[orig_index] == -1)
+								mapping[orig_index] = next_index++;
+
+							boundinfo->indexes[i] = mapping[orig_index];
+						}
+					}
+
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						Assert(default_index >= 0 && mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					boundinfo->indexes[i] = -1;
+					break;
+				}
+
+			default:
+				elog(ERROR, "unexpected partition strategy: %d",
+					 (int) key->strategy);
+		}
+
+		result->boundinfo = boundinfo;
+
+		/*
+		 * Now assign OIDs from the original array into mapped indexes of the
+		 * result array.  Order of OIDs in the former is defined by the
+		 * catalog scan that retrieved them, whereas that in the latter is
+		 * defined by canonicalized representation of the partition bounds.
+		 */
+		for (i = 0; i < nparts; i++)
+			result->oids[mapping[i]] = oids[i];
+		pfree(mapping);
+	}
+
+	MemoryContextSwitchTo(oldcxt);
+	rel->rd_partdesc = result;
+}
+
+/*
+ * Are two partition bound collections logically equal?
+ *
+ * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
+ * This is also useful when b1 and b2 are bound collections of two separate
+ * relations, respectively, because PartitionBoundInfo is a canonical
+ * representation of partition bounds.
+ */
+bool
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
+					   PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			i;
+
+	if (b1->strategy != b2->strategy)
+		return false;
+
+	if (b1->ndatums != b2->ndatums)
+		return false;
+
+	if (b1->null_index != b2->null_index)
+		return false;
+
+	if (b1->default_index != b2->default_index)
+		return false;
+
+	if (b1->strategy == PARTITION_STRATEGY_HASH)
+	{
+		int			greatest_modulus = get_greatest_modulus(b1);
+
+		/*
+		 * If two hash partitioned tables have different greatest moduli,
+		 * their partition schemes don't match.
+		 */
+		if (greatest_modulus != get_greatest_modulus(b2))
+			return false;
+
+		/*
+		 * We arrange the partitions in the ascending order of their modulus
+		 * and remainders.  Also every modulus is factor of next larger
+		 * modulus.  Therefore we can safely store index of a given partition
+		 * in indexes array at remainder of that partition.  Also entries at
+		 * (remainder + N * modulus) positions in indexes array are all same
+		 * for (modulus, remainder) specification for any partition.  Thus
+		 * datums array from both the given bounds are same, if and only if
+		 * their indexes array will be same.  So, it suffices to compare
+		 * indexes array.
+		 */
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+		/*
+		 * Nonetheless make sure that the bounds are indeed same when the
+		 * indexes match.  Hash partition bound stores modulus and remainder
+		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			Assert((b1->datums[i][0] == b2->datums[i][0] &&
+					b1->datums[i][1] == b2->datums[i][1]));
+#endif
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
+		{
+			int			j;
+
+			for (j = 0; j < partnatts; j++)
+			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->kind != NULL)
+				{
+					/* The different kinds of bound all differ from each other */
+					if (b1->kind[i][j] != b2->kind[i][j])
+						return false;
+
+					/*
+					 * Non-finite bounds are equal without further
+					 * examination.
+					 */
+					if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
+						continue;
+				}
+
+				/*
+				 * Compare the actual values. Note that it would be both
+				 * incorrect and unsafe to invoke the comparison operator
+				 * derived from the partitioning specification here.  It would
+				 * be incorrect because we want the relcache entry to be
+				 * updated for ANY change to the partition bounds, not just
+				 * those that the partitioning operator thinks are
+				 * significant.  It would be unsafe because we might reach
+				 * this code in the context of an aborted transaction, and an
+				 * arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be
+				 * safe.
+				 */
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  parttypbyval[j], parttyplen[j]))
+					return false;
+			}
+
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+		}
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (b1->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
+			return false;
+	}
+	return true;
+}
+
+/*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specification.
+ */
+PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo dest;
+	int			i;
+	int			ndatums;
+	int			partnatts;
+	int			num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	num_indexes = get_partition_bound_num_indexes(src);
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+														 sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+															   sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int			j;
+
+		/*
+		 * For a corresponding to hash partition, datums array will have two
+		 * elements - modulus and remainder.
+		 */
+		bool		hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
+		int			natts = hash_part ? 2 : partnatts;
+
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
+
+		for (j = 0; j < natts; j++)
+		{
+			bool		byval;
+			int			typlen;
+
+			if (hash_part)
+			{
+				typlen = sizeof(int32); /* Always int4 */
+				byval = true;	/* int4 is pass-by-value */
+			}
+			else
+			{
+				byval = key->parttypbyval[j];
+				typlen = key->parttyplen[j];
+			}
+
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   byval, typlen);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
+ * check_new_partition_bound
+ *
+ * Checks if the new partition's bound overlaps any of the existing partitions
+ * of parent.  Also performs additional checks as necessary per strategy.
+ */
+void
+check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec)
+{
+	PartitionKey key = RelationGetPartitionKey(parent);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
+	ParseState *pstate = make_parsestate(NULL);
+	int			with = -1;
+	bool		overlap = false;
+
+	if (spec->is_default)
+	{
+		if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+			return;
+
+		/* Default partition already exists, error out. */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+				 parser_errposition(pstate, spec->location)));
+	}
+
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					Datum	  **datums = boundinfo->datums;
+					int			ndatums = boundinfo->ndatums;
+					int			greatest_modulus;
+					int			remainder;
+					int			offset;
+					bool		valid_modulus = true;
+					int			prev_modulus,	/* Previous largest modulus */
+								next_modulus;	/* Next largest modulus */
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.
+					 *
+					 * Get the greatest (modulus, remainder) pair contained in
+					 * boundinfo->datums that is less than or equal to the
+					 * (spec->modulus, spec->remainder) pair.
+					 */
+					offset = partition_hash_bsearch(boundinfo,
+													spec->modulus,
+													spec->remainder);
+					if (offset < 0)
+					{
+						next_modulus = DatumGetInt32(datums[0][0]);
+						valid_modulus = (next_modulus % spec->modulus) == 0;
+					}
+					else
+					{
+						prev_modulus = DatumGetInt32(datums[offset][0]);
+						valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+						if (valid_modulus && (offset + 1) < ndatums)
+						{
+							next_modulus = DatumGetInt32(datums[offset + 1][0]);
+							valid_modulus = (next_modulus % spec->modulus) == 0;
+						}
+					}
+
+					if (!valid_modulus)
+						ereport(ERROR,
+								(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+								 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
+
+					greatest_modulus = get_greatest_modulus(boundinfo);
+					remainder = spec->remainder;
+
+					/*
+					 * Normally, the lowest remainder that could conflict with
+					 * the new partition is equal to the remainder specified
+					 * for the new partition, but when the new partition has a
+					 * modulus higher than any used so far, we need to adjust.
+					 */
+					if (remainder >= greatest_modulus)
+						remainder = remainder % greatest_modulus;
+
+					/* Check every potentially-conflicting remainder. */
+					do
+					{
+						if (boundinfo->indexes[remainder] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[remainder];
+							break;
+						}
+						remainder += spec->modulus;
+					} while (remainder < greatest_modulus);
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_LIST:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+
+				if (partdesc->nparts > 0)
+				{
+					ListCell   *cell;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					foreach(cell, spec->listdatums)
+					{
+						Const	   *val = castNode(Const, lfirst(cell));
+
+						if (!val->constisnull)
+						{
+							int			offset;
+							bool		equal;
+
+							offset = partition_list_bsearch(key->partsupfunc,
+														key->partcollation,
+															boundinfo,
+															val->constvalue,
+															&equal);
+							if (offset >= 0 && equal)
+							{
+								overlap = true;
+								with = boundinfo->indexes[offset];
+								break;
+							}
+						}
+						else if (partition_bound_accepts_nulls(boundinfo))
+						{
+							overlap = true;
+							with = boundinfo->null_index;
+							break;
+						}
+					}
+				}
+
+				break;
+			}
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				PartitionRangeBound *lower,
+						   *upper;
+
+				Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
+				lower = make_one_range_bound(key, -1, spec->lowerdatums, true);
+				upper = make_one_range_bound(key, -1, spec->upperdatums, false);
+
+				/*
+				 * First check if the resulting range would be empty with
+				 * specified lower and upper bounds
+				 */
+				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
+										 key->partcollation, lower->datums,
+										 lower->kind, true, upper) >= 0)
+				{
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("empty range bound specified for partition \"%s\"",
+									relname),
+							 errdetail("Specified lower bound %s is greater than or equal to upper bound %s.",
+									   get_range_partbound_string(spec->lowerdatums),
+									   get_range_partbound_string(spec->upperdatums)),
+							 parser_errposition(pstate, spec->location)));
+				}
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo boundinfo = partdesc->boundinfo;
+					int			offset;
+					bool		equal;
+
+					Assert(boundinfo &&
+						   boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+						   (boundinfo->ndatums > 0 ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Test whether the new lower bound (which is treated
+					 * inclusively as part of the new partition) lies inside
+					 * an existing partition, or in a gap.
+					 *
+					 * If it's inside an existing partition, the bound at
+					 * offset + 1 will be the upper bound of that partition,
+					 * and its index will be >= 0.
+					 *
+					 * If it's in a gap, the bound at offset + 1 will be the
+					 * lower bound of the next partition, and its index will
+					 * be -1. This is also true if there is no next partition,
+					 * since the index array is initialised with an extra -1
+					 * at the end.
+					 */
+					offset = partition_range_bsearch(key->partnatts,
+													 key->partsupfunc,
+													 key->partcollation,
+													 boundinfo, lower,
+													 &equal);
+
+					if (boundinfo->indexes[offset + 1] < 0)
+					{
+						/*
+						 * Check that the new partition will fit in the gap.
+						 * For it to fit, the new upper bound must be less
+						 * than or equal to the lower bound of the next
+						 * partition, if there is one.
+						 */
+						if (offset + 1 < boundinfo->ndatums)
+						{
+							int32		cmpval;
+							Datum 	   *datums;
+							PartitionRangeDatumKind *kind;
+							bool		is_lower;
+
+							datums = boundinfo->datums[offset + 1];
+							kind = boundinfo->kind[offset + 1];
+							is_lower = (boundinfo->indexes[offset + 1] == -1);
+
+							cmpval = partition_rbound_cmp(key->partnatts,
+														  key->partsupfunc,
+														  key->partcollation,
+														  datums, kind,
+														  is_lower, upper);
+							if (cmpval < 0)
+							{
+								/*
+								 * The new partition overlaps with the
+								 * existing partition between offset + 1 and
+								 * offset + 2.
+								 */
+								overlap = true;
+								with = boundinfo->indexes[offset + 2];
+							}
+						}
+					}
+					else
+					{
+						/*
+						 * The new partition overlaps with the existing
+						 * partition between offset and offset + 1.
+						 */
+						overlap = true;
+						with = boundinfo->indexes[offset + 1];
+					}
+				}
+
+				break;
+			}
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	if (overlap)
+	{
+		Assert(with >= 0);
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("partition \"%s\" would overlap partition \"%s\"",
+						relname, get_rel_name(partdesc->oids[with])),
+				 parser_errposition(pstate, spec->location)));
+	}
+}
+
+/*
+ * RelationGetPartitionQual
+ *
+ * Returns a list of partition quals
+ */
+List *
+RelationGetPartitionQual(Relation rel)
+{
+	/* Quick exit */
+	if (!rel->rd_rel->relispartition)
+		return NIL;
+
+	return generate_partition_qual(rel);
+}
+
+/*
+ * get_partition_qual_relid
+ *
+ * Returns an expression tree describing the passed-in relation's partition
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
+ */
+Expr *
+get_partition_qual_relid(Oid relid)
+{
+	Relation	rel = heap_open(relid, AccessShareLock);
+	Expr	   *result = NULL;
+	List	   *and_args;
+
+	/* Do the work only if this relation is a partition. */
+	if (rel->rd_rel->relispartition)
+	{
+		and_args = generate_partition_qual(rel);
+
+		if (and_args == NIL)
+			result = NULL;
+		else if (list_length(and_args) > 1)
+			result = makeBoolExpr(AND_EXPR, and_args, -1);
+		else
+			result = linitial(and_args);
+	}
+
+	/* Keep the lock. */
+	heap_close(rel, NoLock);
+
+	return result;
+}
+
+/*
+ * Checks if any of the 'attnums' is a partition key attribute for rel
+ *
+ * Sets *used_in_expr if any of the 'attnums' is found to be referenced in some
+ * partition key expression.  It's possible for a column to be both used
+ * directly and as part of an expression; if that happens, *used_in_expr may
+ * end up as either true or false.  That's OK for current uses of this
+ * function, because *used_in_expr is only used to tailor the error message
+ * text.
+ */
+bool
+has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr)
+{
+	PartitionKey key;
+	int			partnatts;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	int			i;
+
+	if (attnums == NULL || rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+		return false;
+
+	key = RelationGetPartitionKey(rel);
+	partnatts = get_partition_natts(key);
+	partexprs = get_partition_exprs(key);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		AttrNumber	partattno = get_partition_col_attnum(key, i);
+
+		if (partattno != 0)
+		{
+			if (bms_is_member(partattno - FirstLowInvalidHeapAttributeNumber,
+							  attnums))
+			{
+				if (used_in_expr)
+					*used_in_expr = false;
+				return true;
+			}
+		}
+		else
+		{
+			/* Arbitrary expression */
+			Node	   *expr = (Node *) lfirst(partexprs_item);
+			Bitmapset  *expr_attrs = NULL;
+
+			/* Find all attributes referenced */
+			pull_varattnos(expr, 1, &expr_attrs);
+			partexprs_item = lnext(partexprs_item);
+
+			if (bms_overlap(attnums, expr_attrs))
+			{
+				if (used_in_expr)
+					*used_in_expr = true;
+				return true;
+			}
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple
+ *		Finds partition of relation which accepts the partition key specified
+ *		in values and isnull
+ *
+ * Return value is index of the partition (>= 0 and < partdesc->nparts) if one
+ * found or -1 if none found.
+ */
+int
+get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
+{
+	int			bound_offset;
+	int			part_index = -1;
+	PartitionKey key = RelationGetPartitionKey(relation);
+	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
+
+	/* Route as appropriate based on partitioning strategy. */
+	switch (key->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundInfo boundinfo = partdesc->boundinfo;
+				int			greatest_modulus = get_greatest_modulus(boundinfo);
+				uint64		rowHash = compute_hash_value(key->partnatts,
+														 key->partsupfunc,
+														 values, isnull);
+
+				part_index = boundinfo->indexes[rowHash % greatest_modulus];
+			}
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			if (isnull[0])
+			{
+				if (partition_bound_accepts_nulls(partdesc->boundinfo))
+					part_index = partdesc->boundinfo->null_index;
+			}
+			else
+			{
+				bool		equal = false;
+
+				bound_offset = partition_list_bsearch(key->partsupfunc,
+													  key->partcollation,
+													  partdesc->boundinfo,
+													  values[0], &equal);
+				if (bound_offset >= 0 && equal)
+					part_index = partdesc->boundinfo->indexes[bound_offset];
+			}
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			{
+				bool		equal = false,
+							range_partkey_has_null = false;
+				int			i;
+
+				/*
+				 * No range includes NULL, so this will be accepted by the
+				 * default partition if there is one, and otherwise rejected.
+				 */
+				for (i = 0; i < key->partnatts; i++)
+				{
+					if (isnull[i])
+					{
+						range_partkey_has_null = true;
+						break;
+					}
+				}
+
+				if (!range_partkey_has_null)
+				{
+					bound_offset =
+							partition_range_datum_bsearch(key->partsupfunc,
+														  key->partcollation,
+														  partdesc->boundinfo,
+														  key->partnatts,
+														  values,
+														  &equal);
+					/*
+					 * The bound at bound_offset is less than or equal to the
+					 * tuple value, so the bound at offset+1 is the upper
+					 * bound of the partition we're looking for, if there
+					 * actually exists one.
+					 */
+					part_index = partdesc->boundinfo->indexes[bound_offset + 1];
+				}
+			}
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) key->strategy);
+	}
+
+	/*
+	 * part_index < 0 means we failed to find a partition of this parent. Use
+	 * the default partition, if there is one.
+	 */
+	if (part_index < 0)
+		part_index = partdesc->boundinfo->default_index;
+
+	return part_index;
+}
+
+/*
+ * get_greatest_modulus
+ *
+ * Returns the greatest modulus of the hash partition bound. The greatest
+ * modulus will be at the end of the datums array because hash partitions are
+ * arranged in the ascending order of their modulus and remainders.
+ */
+int
+get_greatest_modulus(PartitionBoundInfo bound)
+{
+	Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
+	Assert(bound->datums && bound->ndatums > 0);
+	Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
+
+	return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+uint64
+compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
+				   Datum *values, bool *isnull)
+{
+	int			i;
+	int			nkeys = partnatts;
+	uint64		rowHash = 0;
+	Datum		seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+	for (i = 0; i < nkeys; i++)
+	{
+		if (!isnull[i])
+		{
+			Datum		hash;
+
+			Assert(OidIsValid(partsupfunc[i].fn_oid));
+
+			/*
+			 * Compute hash for each datum value by calling respective
+			 * datatype-specific hash functions of each partition key
+			 * attribute.
+			 */
+			hash = FunctionCall2(&partsupfunc[i], values[i], seed);
+
+			/* Form a single 64-bit hash value */
+			rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+	if (partdesc && partdesc->boundinfo &&
+		partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
+
+/* Module-local functions. */
+
+/*
+ * generate_partition_qual
+ *
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
+ *
+ * Result expression tree is stored CacheMemoryContext to ensure it survives
+ * as long as the relcache entry. But we should be running in a less long-lived
+ * working context. To avoid leaking cache memory if this routine fails partway
+ * through, we build in working memory and then copy the completed structure
+ * into cache memory.
+ */
+static List *
+generate_partition_qual(Relation rel)
+{
+	HeapTuple	tuple;
+	MemoryContext oldcxt;
+	Datum		boundDatum;
+	bool		isnull;
+	PartitionBoundSpec *bound;
+	List	   *my_qual = NIL,
+			   *result = NIL;
+	Relation	parent;
+	bool		found_whole_row;
+
+	/* Guard against stack overflow due to overly deep partition tree */
+	check_stack_depth();
+
+	/* Quick copy */
+	if (rel->rd_partcheck != NIL)
+		return copyObject(rel->rd_partcheck);
+
+	/* Grab at least an AccessShareLock on the parent table */
+	parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
+					   AccessShareLock);
+
+	/* Get pg_class.relpartbound */
+	tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	boundDatum = SysCacheGetAttr(RELOID, tuple,
+								 Anum_pg_class_relpartbound,
+								 &isnull);
+	if (isnull)					/* should not happen */
+		elog(ERROR, "relation \"%s\" has relpartbound = null",
+			 RelationGetRelationName(rel));
+	bound = castNode(PartitionBoundSpec,
+					 stringToNode(TextDatumGetCString(boundDatum)));
+	ReleaseSysCache(tuple);
+
+	my_qual = get_qual_from_partbound(rel, parent, bound);
+
+	/* 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;
+
+	/*
+	 * Change Vars to have partition's attnos instead of the parent's. We do
+	 * this after we concatenate the parent's quals, because we want every Var
+	 * in it to bear this relation's attnos. It's safe to assume varno = 1
+	 * here.
+	 */
+	result = map_partition_varattnos(result, 1, rel, parent,
+									 &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");
+
+	/* Save a copy in the relcache */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	rel->rd_partcheck = copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* Keep the parent locked until commit */
+	heap_close(parent, NoLock);
+
+	return result;
+}
+
+/*
+ * partition_hbound_cmp
+ *
+ * Compares modulus first, then remainder if modulus are equal.
+ */
+static int32
+partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
+{
+	if (modulus1 < modulus2)
+		return -1;
+	if (modulus1 > modulus2)
+		return 1;
+	if (modulus1 == modulus2 && remainder1 != remainder2)
+		return (remainder1 > remainder2) ? 1 : -1;
+	return 0;
+}
+
+/*
+ * qsort_partition_hbound_cmp
+ *
+ * We sort hash bounds by modulus, then by remainder.
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+
+	return partition_hbound_cmp(h1->modulus, h1->remainder,
+								h2->modulus, h2->remainder);
+}
+
+/*
+ * qsort_partition_list_value_cmp
+ *
+ * Compare two list partition bound datums
+ */
+static int32
+qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
+{
+	Datum		val1 = (*(const PartitionListValue **) a)->value,
+				val2 = (*(const PartitionListValue **) b)->value;
+	PartitionKey key = (PartitionKey) arg;
+
+	return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+										   key->partcollation[0],
+										   val1, val2));
+}
+
+/*
+ * make_one_range_bound
+ *
+ * Return a PartitionRangeBound given a list of PartitionRangeDatum elements
+ * and a flag telling whether the bound is lower or not.  Made into a function
+ * because there are multiple sites that want to use this facility.
+ */
+static PartitionRangeBound *
+make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
+{
+	PartitionRangeBound *bound;
+	ListCell   *lc;
+	int			i;
+
+	Assert(datums != NIL);
+
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
+	bound->index = index;
+	bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
+	bound->kind = (PartitionRangeDatumKind *) palloc0(key->partnatts *
+													  sizeof(PartitionRangeDatumKind));
+	bound->lower = lower;
+
+	i = 0;
+	foreach(lc, datums)
+	{
+		PartitionRangeDatum *datum = castNode(PartitionRangeDatum, lfirst(lc));
+
+		/* What's contained in this range datum? */
+		bound->kind[i] = datum->kind;
+
+		if (datum->kind == PARTITION_RANGE_DATUM_VALUE)
+		{
+			Const	   *val = castNode(Const, datum->value);
+
+			if (val->constisnull)
+				elog(ERROR, "invalid range bound datum");
+			bound->datums[i] = val->constvalue;
+		}
+
+		i++;
+	}
+
+	return bound;
+}
+
+/*
+ * partition_rbound_cmp
+ *
+ * Return for two range bounds whether the 1st one (specified in datums1,
+ * kind1, and lower1) is <, =, or > the bound specified in *b2.
+ *
+ * partnatts, partsupfunc and partcollation give the number of attributes in the
+ * bounds to be compared, comparison function to be used and the collations of
+ * attributes, respectively.
+ *
+ * Note that if the values of the two range bounds compare equal, then we take
+ * into account whether they are upper or lower bounds, and an upper bound is
+ * considered to be smaller than a lower bound. This is important to the way
+ * that RelationBuildPartitionDesc() builds the PartitionBoundInfoData
+ * structure, which only stores the upper bound of a common boundary between
+ * two contiguous partitions.
+ */
+static int32
+partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					 Datum *datums1, PartitionRangeDatumKind *kind1,
+					 bool lower1, PartitionRangeBound *b2)
+{
+	int32		cmpval = 0;		/* placate compiler */
+	int			i;
+	Datum	   *datums2 = b2->datums;
+	PartitionRangeDatumKind *kind2 = b2->kind;
+	bool		lower2 = b2->lower;
+
+	for (i = 0; i < partnatts; i++)
+	{
+		/*
+		 * First, handle cases where the column is unbounded, which should not
+		 * invoke the comparison procedure, and should not consider any later
+		 * columns. Note that the PartitionRangeDatumKind enum elements
+		 * compare the same way as the values they represent.
+		 */
+		if (kind1[i] < kind2[i])
+			return -1;
+		else if (kind1[i] > kind2[i])
+			return 1;
+		else if (kind1[i] != PARTITION_RANGE_DATUM_VALUE)
+
+			/*
+			 * The column bounds are both MINVALUE or both MAXVALUE. No later
+			 * columns should be considered, but we still need to compare
+			 * whether they are upper or lower bounds.
+			 */
+			break;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
+												 datums1[i],
+												 datums2[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	/*
+	 * If the comparison is anything other than equal, we're done. If they
+	 * compare equal though, we still have to consider whether the boundaries
+	 * are inclusive or exclusive.  Exclusive one is considered smaller of the
+	 * two.
+	 */
+	if (cmpval == 0 && lower1 != lower2)
+		cmpval = lower1 ? 1 : -1;
+
+	return cmpval;
+}
+
+/* Used when sorting range bounds across all range partitions */
+static int32
+qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
+{
+	PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
+	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
+	PartitionKey key = (PartitionKey) arg;
+
+	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
+								key->partcollation, b1->datums, b1->kind,
+								b1->lower, b2);
+}
+
+/*
+ * partition_list_bsearch
+ *		Returns the index of the greatest bound datum that is less than equal
+ * 		to the given value or -1 if all of the bound datums are greater
+ *
+ * *is_equal is set to true if the bound datum at the returned index is equal
+ * to the input value.
+ */
+static int
+partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo,
+					   Datum value, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+												 partcollation[0],
+												 boundinfo->datums[mid][0],
+												 value));
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_rbound_datum_cmp
+ *
+ * Return whether range bound (specified in rb_datums, rb_kind, and rb_lower)
+ * is <, =, or > partition key of tuple (tuple_datums)
+ *
+ * n_tuple_datums, partsupfunc and partcollation give number of attributes in
+ * the bounds to be compared, comparison function to be used and the collations
+ * of attributes resp.
+ *
+ */
+static int32
+partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,
+						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+						   Datum *tuple_datums, int n_tuple_datums)
+{
+	int			i;
+	int32		cmpval = -1;
+
+	for (i = 0; i < n_tuple_datums; i++)
+	{
+		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
+			return -1;
+		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
+			return 1;
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
+												 rb_datums[i],
+												 tuple_datums[i]));
+		if (cmpval != 0)
+			break;
+	}
+
+	return cmpval;
+}
+
+/*
+ * partition_range_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given range bound or -1 if all of the range bounds are
+ *		greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input range bound
+ */
+static int
+partition_range_bsearch(int partnatts,
+						FmgrInfo *partsupfunc, Oid *partcollation,
+						PartitionBoundInfo boundinfo,
+						PartitionRangeBound *probe, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_cmp(partnatts,
+									  partsupfunc, partcollation,
+									  boundinfo->datums[mid],
+									  boundinfo->kind[mid],
+									  (boundinfo->indexes[mid] == -1),
+									  probe);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_range_datum_bsearch
+ *		Returns the index of the greatest range bound that is less than or
+ *		equal to the given tuple or -1 if all of the range bounds are greater
+ *
+ * *is_equal is set to true if the range bound at the returned index is equal
+ * to the input tuple.
+ */
+static int
+partition_range_datum_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+							  PartitionBoundInfo boundinfo,
+							  int nvalues, Datum *values, bool *is_equal)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval;
+
+		mid = (lo + hi + 1) / 2;
+		cmpval = partition_rbound_datum_cmp(partsupfunc, partcollation,
+											boundinfo->datums[mid],
+											boundinfo->kind[mid],
+											values,
+											nvalues);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+			*is_equal = (cmpval == 0);
+
+			if (*is_equal)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * partition_hash_bsearch
+ *		Returns the index of the greatest (modulus, remainder) pair that is
+ *		less than or equal to the given (modulus, remainder) pair or -1 if
+ *		all of them are greater
+ */
+static int
+partition_hash_bsearch(PartitionBoundInfo boundinfo,
+					   int modulus, int remainder)
+{
+	int			lo,
+				hi,
+				mid;
+
+	lo = -1;
+	hi = boundinfo->ndatums - 1;
+	while (lo < hi)
+	{
+		int32		cmpval,
+					bound_modulus,
+					bound_remainder;
+
+		mid = (lo + hi + 1) / 2;
+		bound_modulus = DatumGetInt32(boundinfo->datums[mid][0]);
+		bound_remainder = DatumGetInt32(boundinfo->datums[mid][1]);
+		cmpval = partition_hbound_cmp(bound_modulus, bound_remainder,
+									  modulus, remainder);
+		if (cmpval <= 0)
+		{
+			lo = mid;
+
+			if (cmpval == 0)
+				break;
+		}
+		else
+			hi = mid - 1;
+	}
+
+	return lo;
+}
+
+/*
+ * get_partition_bound_num_indexes
+ *
+ * Returns the number of the entries in the partition bound indexes array.
+ */
+static int
+get_partition_bound_num_indexes(PartitionBoundInfo bound)
+{
+	int			num_indexes;
+
+	Assert(bound);
+
+	switch (bound->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * The number of the entries in the indexes array is same as the
+			 * greatest modulus.
+			 */
+			num_indexes = get_greatest_modulus(bound);
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			num_indexes = bound->ndatums;
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			/* Range partitioned table has an extra index. */
+			num_indexes = bound->ndatums + 1;
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) bound->strategy);
+	}
+
+	return num_indexes;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 48f92dc430..f98bc1f921 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -43,7 +43,6 @@
 #include "catalog/index.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_amproc.h"
 #include "catalog/pg_attrdef.h"
@@ -85,6 +84,7 @@
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partcache.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -265,7 +265,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
 static Relation AllocateRelationDesc(Form_pg_class relp);
 static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
 static void RelationBuildTupleDesc(Relation relation);
-static void RelationBuildPartitionKey(Relation relation);
 static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
 static void RelationInitPhysicalAddr(Relation relation);
 static void load_critical_index(Oid indexoid, Oid heapoid);
@@ -874,211 +873,6 @@ RelationBuildRuleLock(Relation relation)
 }
 
 /*
- * RelationBuildPartitionKey
- *		Build and attach to relcache partition key data of relation
- *
- * Partitioning key data is a complex structure; to avoid complicated logic to
- * free individual elements whenever the relcache entry is flushed, we give it
- * its own memory context, child of CacheMemoryContext, which can easily be
- * deleted on its own.  To avoid leaking memory in that context in case of an
- * error partway through this function, the context is initially created as a
- * child of CurTransactionContext and only re-parented to CacheMemoryContext
- * at the end, when no further errors are possible.  Also, we don't make this
- * context the current context except in very brief code sections, out of fear
- * that some of our callees allocate memory on their own which would be leaked
- * permanently.
- */
-static void
-RelationBuildPartitionKey(Relation relation)
-{
-	Form_pg_partitioned_table form;
-	HeapTuple	tuple;
-	bool		isnull;
-	int			i;
-	PartitionKey key;
-	AttrNumber *attrs;
-	oidvector  *opclass;
-	oidvector  *collation;
-	ListCell   *partexprs_item;
-	Datum		datum;
-	MemoryContext partkeycxt,
-				oldcxt;
-	int16		procnum;
-
-	tuple = SearchSysCache1(PARTRELID,
-							ObjectIdGetDatum(RelationGetRelid(relation)));
-
-	/*
-	 * The following happens when we have created our pg_class entry but not
-	 * the pg_partitioned_table entry yet.
-	 */
-	if (!HeapTupleIsValid(tuple))
-		return;
-
-	partkeycxt = AllocSetContextCreate(CurTransactionContext,
-									   "partition key",
-									   ALLOCSET_SMALL_SIZES);
-	MemoryContextCopySetIdentifier(partkeycxt,
-								   RelationGetRelationName(relation));
-
-	key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
-												sizeof(PartitionKeyData));
-
-	/* Fixed-length attributes */
-	form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
-	key->strategy = form->partstrat;
-	key->partnatts = form->partnatts;
-
-	/*
-	 * We can rely on the first variable-length attribute being mapped to the
-	 * relevant field of the catalog's C struct, because all previous
-	 * attributes are non-nullable and fixed-length.
-	 */
-	attrs = form->partattrs.values;
-
-	/* But use the hard way to retrieve further variable-length attributes */
-	/* Operator class */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partclass, &isnull);
-	Assert(!isnull);
-	opclass = (oidvector *) DatumGetPointer(datum);
-
-	/* Collation */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partcollation, &isnull);
-	Assert(!isnull);
-	collation = (oidvector *) DatumGetPointer(datum);
-
-	/* Expressions */
-	datum = SysCacheGetAttr(PARTRELID, tuple,
-							Anum_pg_partitioned_table_partexprs, &isnull);
-	if (!isnull)
-	{
-		char	   *exprString;
-		Node	   *expr;
-
-		exprString = TextDatumGetCString(datum);
-		expr = stringToNode(exprString);
-		pfree(exprString);
-
-		/*
-		 * Run the expressions through const-simplification since the planner
-		 * will be comparing them to similarly-processed qual clause operands,
-		 * and may fail to detect valid matches without this step; fix
-		 * opfuncids while at it.  We don't need to bother with
-		 * canonicalize_qual() though, because partition expressions should be
-		 * in canonical form already (ie, no need for OR-merging or constant
-		 * elimination).
-		 */
-		expr = eval_const_expressions(NULL, expr);
-		fix_opfuncids(expr);
-
-		oldcxt = MemoryContextSwitchTo(partkeycxt);
-		key->partexprs = (List *) copyObject(expr);
-		MemoryContextSwitchTo(oldcxt);
-	}
-
-	oldcxt = MemoryContextSwitchTo(partkeycxt);
-	key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
-	key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
-
-	key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-
-	/* Gather type and collation info as well */
-	key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
-	key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
-	key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
-	key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
-	key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
-	MemoryContextSwitchTo(oldcxt);
-
-	/* determine support function number to search for */
-	procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
-		HASHEXTENDED_PROC : BTORDER_PROC;
-
-	/* Copy partattrs and fill other per-attribute info */
-	memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
-	partexprs_item = list_head(key->partexprs);
-	for (i = 0; i < key->partnatts; i++)
-	{
-		AttrNumber	attno = key->partattrs[i];
-		HeapTuple	opclasstup;
-		Form_pg_opclass opclassform;
-		Oid			funcid;
-
-		/* Collect opfamily information */
-		opclasstup = SearchSysCache1(CLAOID,
-									 ObjectIdGetDatum(opclass->values[i]));
-		if (!HeapTupleIsValid(opclasstup))
-			elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
-
-		opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
-		key->partopfamily[i] = opclassform->opcfamily;
-		key->partopcintype[i] = opclassform->opcintype;
-
-		/* Get a support function for the specified opfamily and datatypes */
-		funcid = get_opfamily_proc(opclassform->opcfamily,
-								   opclassform->opcintype,
-								   opclassform->opcintype,
-								   procnum);
-		if (!OidIsValid(funcid))
-			ereport(ERROR,
-					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
-					 errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
-							NameStr(opclassform->opcname),
-							(key->strategy == PARTITION_STRATEGY_HASH) ?
-							"hash" : "btree",
-							procnum,
-							format_type_be(opclassform->opcintype))));
-
-		fmgr_info(funcid, &key->partsupfunc[i]);
-
-		/* Collation */
-		key->partcollation[i] = collation->values[i];
-
-		/* Collect type information */
-		if (attno != 0)
-		{
-			Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
-
-			key->parttypid[i] = att->atttypid;
-			key->parttypmod[i] = att->atttypmod;
-			key->parttypcoll[i] = att->attcollation;
-		}
-		else
-		{
-			if (partexprs_item == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			key->parttypid[i] = exprType(lfirst(partexprs_item));
-			key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
-			key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
-
-			partexprs_item = lnext(partexprs_item);
-		}
-		get_typlenbyvalalign(key->parttypid[i],
-							 &key->parttyplen[i],
-							 &key->parttypbyval[i],
-							 &key->parttypalign[i]);
-
-		ReleaseSysCache(opclasstup);
-	}
-
-	ReleaseSysCache(tuple);
-
-	/*
-	 * Success --- reparent our context and make the relcache point to the
-	 * newly constructed key
-	 */
-	MemoryContextSetParent(partkeycxt, CacheMemoryContext);
-	relation->rd_partkeycxt = partkeycxt;
-	relation->rd_partkey = key;
-}
-
-/*
  *		equalRuleLocks
  *
  *		Determine whether two RuleLocks are equivalent
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index cd15faa7a1..d04633c59e 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -14,43 +14,12 @@
 #define PARTITION_H
 
 #include "fmgr.h"
-#include "executor/tuptable.h"
-#include "nodes/execnodes.h"
-#include "parser/parse_node.h"
+#include "nodes/parsenodes.h"
 #include "utils/rel.h"
 
 /* Seed for the extended hash function */
 #define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
 
-/*
- * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
- * associated with partitioned tables as part of its partition descriptor.
- *
- * The internal structure is opaque outside partition.c.
- */
-typedef struct PartitionBoundInfoData *PartitionBoundInfo;
-
-/*
- * Information about partitions of a partitioned table.
- */
-typedef struct PartitionDescData
-{
-	int			nparts;			/* Number of partitions */
-	Oid		   *oids;			/* OIDs of partitions */
-	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
-} PartitionDescData;
-
-typedef struct PartitionDescData *PartitionDesc;
-
-extern void RelationBuildPartitionDesc(Relation relation);
-extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
-					   bool *parttypbyval, PartitionBoundInfo b1,
-					   PartitionBoundInfo b2);
-extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
-					  PartitionKey key);
-
-extern void check_new_partition_bound(char *relname, Relation parent,
-						  PartitionBoundSpec *spec);
 extern Oid	get_partition_parent(Oid relid);
 extern List *get_partition_ancestors(Oid relid);
 extern List *get_qual_from_partbound(Relation rel, Relation parent,
@@ -58,20 +27,11 @@ extern List *get_qual_from_partbound(Relation rel, Relation parent,
 extern List *map_partition_varattnos(List *expr, int fromrel_varno,
 						Relation to_rel, Relation from_rel,
 						bool *found_whole_row);
-extern List *RelationGetPartitionQual(Relation rel);
-extern Expr *get_partition_qual_relid(Oid relid);
-extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
-					bool *used_in_expr);
 
-extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
 extern Oid	get_default_partition_oid(Oid parentId);
 extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
-extern int get_partition_for_tuple(Relation relation, Datum *values,
-						bool *isnull);
-
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 03a599ad57..bd995cb2da 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -13,10 +13,10 @@
 #ifndef EXECPARTITION_H
 #define EXECPARTITION_H
 
-#include "catalog/partition.h"
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "utils/partcache.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
diff --git a/src/include/utils/partcache.h b/src/include/utils/partcache.h
new file mode 100644
index 0000000000..0588e32a67
--- /dev/null
+++ b/src/include/utils/partcache.h
@@ -0,0 +1,187 @@
+/*-------------------------------------------------------------------------
+ *
+ * partcache.h
+ *		Header file for partitioning related cached data structures and
+ *		manipulation functions
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partcache.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTCACHE_H
+#define PARTCACHE_H
+
+#include "fmgr.h"
+#include "utils/relcache.h"
+
+/*
+ * Information about the partition key of a relation
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partitioning strategy */
+	int16		partnatts;		/* number of columns in the partition key */
+	AttrNumber *partattrs;		/* attribute numbers of columns in the
+								 * partition key */
+	List	   *partexprs;		/* list of expressions in the partitioning
+								 * key, or NIL */
+
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+
+	/* Partitioning collation per attribute */
+	Oid		   *partcollation;
+
+	/* Type information per attribute */
+	Oid		   *parttypid;
+	int32	   *parttypmod;
+	int16	   *parttyplen;
+	bool	   *parttypbyval;
+	char	   *parttypalign;
+	Oid		   *parttypcoll;
+}			PartitionKeyData;
+
+typedef struct PartitionKeyData *PartitionKey;
+
+typedef struct PartitionBoundInfoData *PartitionBoundInfo;
+
+/*
+ * Information about partitions of a partitioned table.
+ */
+typedef struct PartitionDescData
+{
+	int			nparts;			/* Number of partitions */
+	Oid		   *oids;			/* OIDs of partitions */
+	PartitionBoundInfo boundinfo;	/* collection of partition bounds */
+} PartitionDescData;
+
+typedef struct PartitionDescData *PartitionDesc;
+
+/*
+ * Information about bounds of a partitioned relation
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+	char		strategy;		/* hash, list or range? */
+	int			ndatums;		/* Length of the datums following array */
+	Datum	  **datums;
+	PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+									 * NULL for hash and list partitioned
+									 * tables */
+	int		   *indexes;		/* Partition indexes */
+	int			null_index;		/* Index of the null-accepting partition; -1
+								 * if there isn't one */
+	int			default_index;	/* Index of the default partition; -1 if there
+								 * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * PartitionKey inquiry functions
+ */
+static inline int
+get_partition_strategy(PartitionKey key)
+{
+	return key->strategy;
+}
+
+static inline int
+get_partition_natts(PartitionKey key)
+{
+	return key->partnatts;
+}
+
+static inline List *
+get_partition_exprs(PartitionKey key)
+{
+	return key->partexprs;
+}
+
+/*
+ * PartitionKey inquiry functions - one column
+ */
+static inline int16
+get_partition_col_attnum(PartitionKey key, int col)
+{
+	return key->partattrs[col];
+}
+
+static inline Oid
+get_partition_col_typid(PartitionKey key, int col)
+{
+	return key->parttypid[col];
+}
+
+static inline int32
+get_partition_col_typmod(PartitionKey key, int col)
+{
+	return key->parttypmod[col];
+}
+
+extern void RelationBuildPartitionKey(Relation relation);
+extern void RelationBuildPartitionDesc(Relation relation);
+extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
+					   bool *parttypbyval, PartitionBoundInfo b1,
+					   PartitionBoundInfo b2);
+
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
+
+extern void check_new_partition_bound(char *relname, Relation parent,
+						  PartitionBoundSpec *spec);
+
+extern List *RelationGetPartitionQual(Relation rel);
+extern Expr *get_partition_qual_relid(Oid relid);
+
+extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
+					bool *used_in_expr);
+
+extern Oid	get_default_oid_from_partdesc(PartitionDesc partdesc);
+
+extern int get_greatest_modulus(PartitionBoundInfo b);
+extern uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
+						Datum *values, bool *isnull);
+
+/* For tuple routing */
+extern int get_partition_for_tuple(Relation relation, Datum *values,
+						bool *isnull);
+
+#endif							/* PARTCACHE_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index c26c395b0b..97472ce22c 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -19,12 +19,11 @@
 #include "catalog/pg_class.h"
 #include "catalog/pg_index.h"
 #include "catalog/pg_publication.h"
-#include "fmgr.h"
 #include "nodes/bitmapset.h"
 #include "rewrite/prs2lock.h"
 #include "storage/block.h"
 #include "storage/relfilenode.h"
-#include "utils/relcache.h"
+#include "utils/partcache.h"
 #include "utils/reltrigger.h"
 
 
@@ -47,36 +46,6 @@ typedef struct LockInfoData
 typedef LockInfoData *LockInfo;
 
 /*
- * Information about the partition key of a relation
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partitioning strategy */
-	int16		partnatts;		/* number of columns in the partition key */
-	AttrNumber *partattrs;		/* attribute numbers of columns in the
-								 * partition key */
-	List	   *partexprs;		/* list of expressions in the partitioning
-								 * key, or NIL */
-
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-
-	/* Partitioning collation per attribute */
-	Oid		   *partcollation;
-
-	/* Type information per attribute */
-	Oid		   *parttypid;
-	int32	   *parttypmod;
-	int16	   *parttyplen;
-	bool	   *parttypbyval;
-	char	   *parttypalign;
-	Oid		   *parttypcoll;
-}			PartitionKeyData;
-
-typedef struct PartitionKeyData *PartitionKey;
-
-/*
  * Here are the contents of a relation cache entry.
  */
 
@@ -603,48 +572,6 @@ typedef struct ViewOptions
 #define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
 
 /*
- * PartitionKey inquiry functions
- */
-static inline int
-get_partition_strategy(PartitionKey key)
-{
-	return key->strategy;
-}
-
-static inline int
-get_partition_natts(PartitionKey key)
-{
-	return key->partnatts;
-}
-
-static inline List *
-get_partition_exprs(PartitionKey key)
-{
-	return key->partexprs;
-}
-
-/*
- * PartitionKey inquiry functions - one column
- */
-static inline int16
-get_partition_col_attnum(PartitionKey key, int col)
-{
-	return key->partattrs[col];
-}
-
-static inline Oid
-get_partition_col_typid(PartitionKey key, int col)
-{
-	return key->parttypid[col];
-}
-
-static inline int32
-get_partition_col_typmod(PartitionKey key, int col)
-{
-	return key->parttypmod[col];
-}
-
-/*
  * RelationGetPartitionDesc
  *		Returns partition descriptor for a relation.
  */
-- 
2.11.0

#20Robert Haas
robertmhaas@gmail.com
In reply to: Amit Langote (#19)
Re: reorganizing partitioning code

On Wed, Mar 28, 2018 at 12:07 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/03/22 11:45, Amit Langote wrote:

FWIW, I did manage to rebase it this morning and posting it here.

Rebased again.

I started wondering if we should separate out stuff related to partition
bounds. That is create a utils/partbound.h and put PartitionBoundInfo and
related comparison and search functions into a utils/adt/partbound.c. I
had started thinking about that when I looked at the code added by the
patch submitted on the "advanced partition matching algorithm for
partition-wise join" thread [1]. I haven't done anything about that though.

adt = Abstract Data Type, which I think we've interpreted up until now
to mean an SQL-visible data type, so that seems like an odd choice.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company