diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index 7479d40..6322ad3 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -900,6 +900,7 @@ extractRelOptions(HeapTuple tuple, TupleDesc tupdesc, Oid amoptions)
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
 		case RELKIND_MATVIEW:
+		case RELKIND_PARTITIONED_REL:
 			options = heap_reloptions(classForm->relkind, datum, false);
 			break;
 		case RELKIND_VIEW:
@@ -1350,6 +1351,7 @@ heap_reloptions(char relkind, Datum reloptions, bool validate)
 			return (bytea *) rdopts;
 		case RELKIND_RELATION:
 		case RELKIND_MATVIEW:
+		case RELKIND_PARTITIONED_REL:
 			return default_reloptions(reloptions, validate, RELOPT_KIND_HEAP);
 		default:
 			/* other relkinds are not supported */
diff --git a/src/backend/catalog/Makefile b/src/backend/catalog/Makefile
index 25130ec..1d3db26 100644
--- a/src/backend/catalog/Makefile
+++ b/src/backend/catalog/Makefile
@@ -14,7 +14,7 @@ OBJS = catalog.o dependency.o heap.o index.o indexing.o namespace.o aclchk.o \
        objectaccess.o objectaddress.o pg_aggregate.o pg_collation.o \
        pg_constraint.o pg_conversion.o \
        pg_depend.o pg_enum.o pg_inherits.o pg_largeobject.o pg_namespace.o \
-       pg_operator.o pg_proc.o pg_range.o pg_db_role_setting.o pg_shdepend.o \
+       pg_operator.o pg_partition.o pg_proc.o pg_range.o pg_db_role_setting.o pg_shdepend.o \
        pg_type.o storage.o toasting.o
 
 BKIFILES = postgres.bki postgres.description postgres.shdescription
@@ -41,8 +41,8 @@ POSTGRES_BKI_SRCS = $(addprefix $(top_srcdir)/src/include/catalog/,\
 	pg_foreign_data_wrapper.h pg_foreign_server.h pg_user_mapping.h \
 	pg_foreign_table.h pg_policy.h pg_replication_origin.h \
 	pg_default_acl.h pg_seclabel.h pg_shseclabel.h \
-	pg_collation.h pg_range.h pg_transform.h \
-	toasting.h indexing.h \
+	pg_collation.h pg_range.h pg_transform.h toasting.h indexing.h \
+	pg_partitioned_rel.h pg_partition.h\
     )
 
 # location of Catalog.pm
diff --git a/src/backend/catalog/aclchk.c b/src/backend/catalog/aclchk.c
index 50a00cf..062dad3 100644
--- a/src/backend/catalog/aclchk.c
+++ b/src/backend/catalog/aclchk.c
@@ -751,6 +751,8 @@ objectsInSchemaToOids(GrantObjectType objtype, List *nspnames)
 			case ACL_OBJECT_RELATION:
 				objs = getRelationsInNamespace(namespaceId, RELKIND_RELATION);
 				objects = list_concat(objects, objs);
+				objs = getRelationsInNamespace(namespaceId, RELKIND_PARTITIONED_REL);
+				objects = list_concat(objects, objs);
 				objs = getRelationsInNamespace(namespaceId, RELKIND_VIEW);
 				objects = list_concat(objects, objs);
 				objs = getRelationsInNamespace(namespaceId, RELKIND_MATVIEW);
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index d04e94d..98f9251 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -47,6 +47,9 @@
 #include "catalog/pg_foreign_table.h"
 #include "catalog/pg_inherits.h"
 #include "catalog/pg_namespace.h"
+#include "catalog/pg_opclass.h"
+#include "catalog/pg_partition.h"
+#include "catalog/pg_partitioned_rel.h"
 #include "catalog/pg_statistic.h"
 #include "catalog/pg_tablespace.h"
 #include "catalog/pg_type.h"
@@ -55,8 +58,11 @@
 #include "catalog/storage_xlog.h"
 #include "commands/tablecmds.h"
 #include "commands/typecmds.h"
+#include "executor/executor.h"
 #include "miscadmin.h"
+#include "nodes/execnodes.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/parsenodes.h"
 #include "optimizer/var.h"
 #include "parser/parse_coerce.h"
 #include "parser/parse_collate.h"
@@ -69,7 +75,9 @@
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
+#include "utils/partition.h"
 #include "utils/rel.h"
+#include "utils/relcache.h"
 #include "utils/ruleutils.h"
 #include "utils/snapmgr.h"
 #include "utils/syscache.h"
@@ -294,6 +302,14 @@ heap_create(const char *relname,
 			 */
 			reltablespace = InvalidOid;
 			break;
+		case RELKIND_PARTITIONED_REL:
+			/*
+			 * Separated from the above because a partitioned table still has
+			 * partitions with storage and will need to be in some tablespace,
+			 * though theirs can override the parent's.
+			 */
+			create_storage = false;
+			break;
 		case RELKIND_SEQUENCE:
 			create_storage = true;
 
@@ -803,6 +819,7 @@ InsertPgClassTuple(Relation pg_class_desc,
 	values[Anum_pg_class_relhastriggers - 1] = BoolGetDatum(rd_rel->relhastriggers);
 	values[Anum_pg_class_relrowsecurity - 1] = BoolGetDatum(rd_rel->relrowsecurity);
 	values[Anum_pg_class_relhassubclass - 1] = BoolGetDatum(rd_rel->relhassubclass);
+	values[Anum_pg_class_relispartition - 1] = BoolGetDatum(rd_rel->relispartition);
 	values[Anum_pg_class_relispopulated - 1] = BoolGetDatum(rd_rel->relispopulated);
 	values[Anum_pg_class_relreplident - 1] = CharGetDatum(rd_rel->relreplident);
 	values[Anum_pg_class_relfrozenxid - 1] = TransactionIdGetDatum(rd_rel->relfrozenxid);
@@ -861,6 +878,7 @@ AddNewRelationTuple(Relation pg_class_desc,
 	switch (relkind)
 	{
 		case RELKIND_RELATION:
+		case RELKIND_PARTITIONED_REL:
 		case RELKIND_MATVIEW:
 		case RELKIND_INDEX:
 		case RELKIND_TOASTVALUE:
@@ -885,6 +903,7 @@ AddNewRelationTuple(Relation pg_class_desc,
 
 	/* Initialize relfrozenxid and relminmxid */
 	if (relkind == RELKIND_RELATION ||
+		relkind == RELKIND_PARTITIONED_REL ||
 		relkind == RELKIND_MATVIEW ||
 		relkind == RELKIND_TOASTVALUE)
 	{
@@ -1099,9 +1118,10 @@ heap_create_with_catalog(const char *relname,
 	{
 		/* Use binary-upgrade override for pg_class.oid/relfilenode? */
 		if (IsBinaryUpgrade &&
-			(relkind == RELKIND_RELATION || relkind == RELKIND_SEQUENCE ||
-			 relkind == RELKIND_VIEW || relkind == RELKIND_MATVIEW ||
-			 relkind == RELKIND_COMPOSITE_TYPE || relkind == RELKIND_FOREIGN_TABLE))
+			(relkind == RELKIND_RELATION || relkind == RELKIND_PARTITIONED_REL ||
+			 relkind == RELKIND_SEQUENCE || relkind == RELKIND_VIEW ||
+			 relkind == RELKIND_MATVIEW || relkind == RELKIND_COMPOSITE_TYPE ||
+			 relkind == RELKIND_FOREIGN_TABLE))
 		{
 			if (!OidIsValid(binary_upgrade_next_heap_pg_class_oid))
 				ereport(ERROR,
@@ -1132,6 +1152,7 @@ heap_create_with_catalog(const char *relname,
 		switch (relkind)
 		{
 			case RELKIND_RELATION:
+			case RELKIND_PARTITIONED_REL:
 			case RELKIND_VIEW:
 			case RELKIND_MATVIEW:
 			case RELKIND_FOREIGN_TABLE:
@@ -1176,6 +1197,7 @@ heap_create_with_catalog(const char *relname,
 	 * such is an implementation detail: toast tables, sequences and indexes.
 	 */
 	if (IsUnderPostmaster && (relkind == RELKIND_RELATION ||
+							  relkind == RELKIND_PARTITIONED_REL ||
 							  relkind == RELKIND_VIEW ||
 							  relkind == RELKIND_MATVIEW ||
 							  relkind == RELKIND_FOREIGN_TABLE ||
@@ -1348,7 +1370,11 @@ heap_create_with_catalog(const char *relname,
 	if (oncommit != ONCOMMIT_NOOP)
 		register_on_commit_action(relid, oncommit);
 
-	if (relpersistence == RELPERSISTENCE_UNLOGGED)
+	/*
+	 * Do not bother creating the init fork for a partitioned relation.
+	 */
+	if (relpersistence == RELPERSISTENCE_UNLOGGED
+				&& relkind != RELKIND_PARTITIONED_REL)
 	{
 		Assert(relkind == RELKIND_RELATION || relkind == RELKIND_MATVIEW ||
 			   relkind == RELKIND_TOASTVALUE);
@@ -1798,11 +1824,29 @@ heap_drop_with_catalog(Oid relid)
 	}
 
 	/*
+	 * Delete the pg_partitioned_rel tuple first.
+	 */
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		RemovePartitionKeyByRelId(relid);
+
+	/*
+	 * The pg_partition entry is otherwise removed only when the partition
+	 * is detached from its parent table. There is no dependency subsystem-
+	 * based tracking of such an entry. Usually, a user command reaching
+	 * this would be a user dropping the whole partitioned table which in
+	 * turn would cascade to its partitions which take with them their
+	 * pg_partition entries, which is what the following is.
+	 */
+	if (rel->rd_rel->relispartition)
+		RemovePartitionBoundByRelId(relid);
+
+	/*
 	 * Schedule unlinking of the relation's physical files at commit.
 	 */
 	if (rel->rd_rel->relkind != RELKIND_VIEW &&
 		rel->rd_rel->relkind != RELKIND_COMPOSITE_TYPE &&
-		rel->rd_rel->relkind != RELKIND_FOREIGN_TABLE)
+		rel->rd_rel->relkind != RELKIND_FOREIGN_TABLE &&
+		rel->rd_rel->relkind != RELKIND_PARTITIONED_REL)
 	{
 		RelationDropStorage(rel);
 	}
@@ -2990,3 +3034,409 @@ insert_ordered_unique_oid(List *list, Oid datum)
 	lappend_cell_oid(list, prev, datum);
 	return list;
 }
+
+/*
+ * StorePartitionKey
+ *
+ * Store the partition key of relation rel into system catalog
+ * pg_partitioned_rel.
+ */
+void
+StorePartitionKey(Relation rel, PartitionKeyInfo *pkinfo, Oid *partOpClassOids,
+					char strategy)
+{
+	int i;
+	int2vector *partkey;
+	oidvector  *partclass;
+	Datum		values[Natts_pg_partitioned_rel];
+	bool		nulls[Natts_pg_partitioned_rel];
+	Datum		partexprsDatum;
+	Relation	pg_partitioned_rel;
+	HeapTuple	tuple;
+	ObjectAddress   myself;
+	ObjectAddress   referenced;
+
+	/* Cannot happen but might as well. */
+	tuple = SearchSysCache(PARTITIONEDRELID,
+							ObjectIdGetDatum(RelationGetRelid(rel)), 0, 0, 0);
+	if (HeapTupleIsValid(tuple))
+	{
+		ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					errmsg("multiple partition keys for table \"%s\" are not allowed",
+					RelationGetRelationName(rel))));
+		ReleaseSysCache(tuple);
+	}
+
+	/*
+	 * Copy the partition key, opclass info into arrays (should we
+	 * make the caller pass them like this to start with?)
+	 */
+	partkey = buildint2vector(pkinfo->pi_KeyAttrNumbers, pkinfo->pi_NumKeyAttrs);
+	partclass = buildoidvector(partOpClassOids, pkinfo->pi_NumKeyAttrs);
+
+	/* Convert the partition key expressions (if any) to a text datum. */
+	if (pkinfo->pi_Expressions)
+	{
+		char       *exprsString;
+
+		exprsString = nodeToString(pkinfo->pi_Expressions);
+		partexprsDatum = CStringGetTextDatum(exprsString);
+		pfree(exprsString);
+	}
+	else
+		partexprsDatum = (Datum) 0;
+
+	pg_partitioned_rel = heap_open(PartitionedRelRelationId, RowExclusiveLock);
+
+	/* Build a pg_partitioned_rel tuple. */
+	MemSet(nulls, false, sizeof(nulls));
+
+	/* Only this can be NULL. */
+	if (!partexprsDatum)
+		nulls[Anum_pg_partitioned_rel_partexprs - 1] = true;
+
+	values[Anum_pg_partitioned_rel_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel));
+	values[Anum_pg_partitioned_rel_partstrategy - 1] = CharGetDatum(strategy);
+	values[Anum_pg_partitioned_rel_partnatts - 1] = Int16GetDatum(pkinfo->pi_NumKeyAttrs);
+	values[Anum_pg_partitioned_rel_partkey - 1] =  PointerGetDatum(partkey);
+	values[Anum_pg_partitioned_rel_partclass - 1] = PointerGetDatum(partclass);
+	values[Anum_pg_partitioned_rel_partexprs - 1] = partexprsDatum;
+
+	tuple = heap_form_tuple(RelationGetDescr(pg_partitioned_rel), values, nulls);
+
+	simple_heap_insert(pg_partitioned_rel, tuple);
+
+	/* Update the indexes on pg_partitioned_rel. */
+	CatalogUpdateIndexes(pg_partitioned_rel, tuple);
+
+	/*
+	 * Store dependencies.
+	 *
+	 * Make this relation dependent on a few things:
+	 */
+	myself.classId = RelationRelationId;
+	myself.objectId = RelationGetRelid(rel);;
+	myself.objectSubId = 0;
+
+	/*
+	 * 1) Operator class per key column.
+	 */
+	for (i = 0; i < pkinfo->pi_NumKeyAttrs; i++)
+	{
+		referenced.classId = OperatorClassRelationId;
+		referenced.objectId = partOpClassOids[i];
+		referenced.objectSubId = 0;
+
+		recordDependencyOn(&myself, &referenced, DEPENDENCY_NORMAL);
+	}
+
+	/*
+	 * 2) Things inside key expressions.
+	 *
+	 * XXX - an ugliness: normal dependencies also created on attribute
+	 * references.
+	 */
+	if (pkinfo->pi_Expressions)
+	{
+		recordDependencyOnSingleRelExpr(&myself,
+										(Node *) pkinfo->pi_Expressions,
+										RelationGetRelid(rel),
+										DEPENDENCY_NORMAL,
+										DEPENDENCY_NORMAL);
+	}
+
+	/* Tell the world about the key. */
+	CacheInvalidateRelcache(rel);
+
+	/* Close the catalog relation and free the tuple. */
+	heap_close(pg_partitioned_rel, RowExclusiveLock);
+	heap_freetuple(tuple);
+}
+
+/*
+ *  RemovePartitionKeyByRelId
+ *
+ *  Remove a pg_partitioned_rel entry.
+ */
+void
+RemovePartitionKeyByRelId(Oid relid)
+{
+	Relation		pkeyrel;
+	HeapTuple		tuple;
+
+	pkeyrel = heap_open(PartitionedRelRelationId, RowExclusiveLock);
+
+	tuple = SearchSysCache1(PARTITIONEDRELID, ObjectIdGetDatum(relid));
+
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for partition key of relation %u", relid);
+
+	simple_heap_delete(pkeyrel, &tuple->t_self);
+
+	ReleaseSysCache(tuple);
+	heap_close(pkeyrel, RowExclusiveLock);
+}
+
+/* ----------------
+ *	BuildPartitionKeyInfo
+ *		Construct a PartitionKeyInfo record for an open relation
+ *
+ * PartitionKeyInfo stores the information about the partition key that's
+ * needed when inserting tuples into a partitioned relations; especially
+ * partition expression state. Normally we build a PartitionKeyInfo for a
+ * partitioned relation just once per command, and then use it for
+ * (potentially) many tuples.
+ * ----------------
+ */
+PartitionKeyInfo *
+BuildPartitionKeyInfo(Relation rel)
+{
+	int							i;
+	int							numKeys;
+	Form_pg_partitioned_rel		pkeyStruct = rel->rd_pkey;
+	PartitionKeyInfo		   *pi = makeNode(PartitionKeyInfo);
+
+	/*
+	 * check the number of keys, and copy attr numbers into the result
+	 */
+	numKeys = pkeyStruct->partnatts;
+	if (numKeys < 1 || numKeys > PARTITION_MAX_KEYS)
+		elog(ERROR, "invalid partnatts %d for partitioned relation %u",
+			 numKeys, RelationGetRelid(rel));
+
+	pi->pi_NumKeyAttrs = numKeys;
+	for (i = 0; i < numKeys; i++)
+		pi->pi_KeyAttrNumbers[i] = pkeyStruct->partkey.values[i];
+
+	/* fetch any expressions needed for expressional partition attributes */
+	pi->pi_Expressions = RelationGetPartitionExpressions(rel);
+	pi->pi_ExpressionsState = NIL;
+
+	return pi;
+}
+
+/*
+ * StorePartitionBounds
+ *
+ * Store partition bounds of relation rel into system catalog
+ * pg_partition
+ */
+void
+StorePartitionBounds(Relation childrel,
+				Relation parentrel,
+				int listnvalues,
+				Datum *datum)
+{
+	Relation		pg_partition;
+	HeapTuple		tuple;
+	ArrayType	   *listvalues = NULL;
+	ArrayType	   *rangebounds = NULL;
+	char			partstrategy;
+	int				partnatts;
+
+	Datum			values[Natts_pg_partition];
+	bool			nulls[Natts_pg_partition];
+
+	PartitionKeyTypeInfo *typinfo = get_key_type_info(parentrel);
+
+	partstrategy = parentrel->rd_pkey->partstrategy;
+	partnatts = parentrel->rd_pkey->partnatts;
+
+	/* Enforced by DefineRelation(). */
+	Assert(partstrategy != PARTITION_STRAT_LIST || partnatts == 1);
+
+	/* Cannot happen but might as well. */
+	tuple = SearchSysCache(PARTITIONID,
+							ObjectIdGetDatum(RelationGetRelid(childrel)),
+							0, 0, 0);
+
+	if (HeapTupleIsValid(tuple))
+	{
+		ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					errmsg("redefining partition bounds for relation \"%s\" is not allowed",
+								RelationGetRelationName(childrel))));
+		ReleaseSysCache(tuple);
+	}
+
+	pg_partition = heap_open(PartitionRelationId, RowExclusiveLock);
+
+	/* Build a pg_partition tuple. */
+	MemSet(nulls, false, sizeof(nulls));
+
+	switch (partstrategy)
+	{
+		case PARTITION_STRAT_LIST:
+			/* A simple 1-D array of values for the only partition column. */
+			listvalues = construct_array(datum,
+										 listnvalues,
+										 typinfo->typid[0],
+										 typinfo->typlen[0],
+										 typinfo->typbyval[0],
+										 typinfo->typalign[0]);
+			/* No range bounds. */
+			nulls[Anum_pg_partition_partrangebounds - 1] = true;
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			/* A 2-D array of range bounds per partition column. */
+			rangebounds = construct_array(datum,
+										 partnatts,
+										 ANYARRAYOID,
+										 -1,
+										 false,
+										 'd');
+
+			/* No list values. */
+			nulls[Anum_pg_partition_partlistvalues - 1] = true;
+			break;
+	}
+
+	values[Anum_pg_partition_partitionid - 1] = ObjectIdGetDatum(RelationGetRelid(childrel));
+	values[Anum_pg_partition_partparent - 1] = ObjectIdGetDatum(RelationGetRelid(parentrel));
+	values[Anum_pg_partition_partlistvalues - 1] = PointerGetDatum(listvalues);
+	values[Anum_pg_partition_partrangebounds - 1] = PointerGetDatum(rangebounds);
+
+	tuple = heap_form_tuple(RelationGetDescr(pg_partition), values, nulls);
+
+	simple_heap_insert(pg_partition, tuple);
+
+	/* Update the indexes on pg_partition. */
+	CatalogUpdateIndexes(pg_partition, tuple);
+
+	/* Close the relation and free the tuple. */
+	heap_close(pg_partition, RowExclusiveLock);
+	heap_freetuple(tuple);
+	pfree(typinfo);
+}
+
+/*
+ *  RemovePartitionBoundByRelId
+ *
+ *  Remove a pg_partition entry for a partition.
+ */
+void
+RemovePartitionBoundByRelId(Oid relid)
+{
+	Relation		partrel;
+	HeapTuple		tuple;
+
+	partrel = heap_open(PartitionRelationId, RowExclusiveLock);
+	tuple = SearchSysCache1(PARTITIONID, ObjectIdGetDatum(relid));
+
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for partition %u", relid);
+
+	simple_heap_delete(partrel, &tuple->t_self);
+
+	ReleaseSysCache(tuple);
+	heap_close(partrel, RowExclusiveLock);
+}
+
+/*
+ * SetRelationIsPartition
+ *
+ * Update relation's pg_class.relispartition.
+ *
+ * Caller had better hold exclusive lock on the relation.
+ *
+ * An important side effect is that a SI update message will be sent out for
+ * the pg_class tuple, which will force other backends to rebuild their
+ * relcache entries for the rel.  Also, this backend will rebuild its
+ * own relcache entry at the next CommandCounterIncrement.
+ */
+void
+SetRelationIsPartition(Relation rel, bool ispartition)
+{
+	Relation	relrel;
+	HeapTuple	reltup;
+	Form_pg_class relStruct;
+
+	relrel = heap_open(RelationRelationId, RowExclusiveLock);
+	reltup = SearchSysCacheCopy1(RELOID,
+								 ObjectIdGetDatum(RelationGetRelid(rel)));
+	if (!HeapTupleIsValid(reltup))
+		elog(ERROR, "cache lookup failed for relation %u",
+			 RelationGetRelid(rel));
+
+	relStruct = (Form_pg_class) GETSTRUCT(reltup);
+
+	if (relStruct->relispartition != ispartition)
+	{
+		relStruct->relispartition = ispartition;
+
+		simple_heap_update(relrel, &reltup->t_self, reltup);
+
+		/* keep catalog indexes current */
+		CatalogUpdateIndexes(relrel, reltup);
+	}
+	else
+	{
+		/* Skip the disk update, but force relcache inval anyway */
+		CacheInvalidateRelcache(rel);
+	}
+
+	heap_freetuple(reltup);
+	heap_close(relrel, RowExclusiveLock);
+}
+
+/*
+ * FormPartitionKeyDatum - Construct values[] and isnull[] arrays for
+ * partition key columns off tuple.
+ */
+void
+FormPartitionKeyDatum(PartitionKeyInfo *pkinfo,
+								TupleTableSlot *slot,
+								EState *estate,
+								Datum *values,
+								bool *isnull)
+{
+	ListCell   *partexpr_item;
+	int			i;
+
+	if (pkinfo->pi_Expressions != NIL &&
+		pkinfo->pi_ExpressionsState == NIL)
+	{
+		/* First time through, set up expression evaluation state */
+		pkinfo->pi_ExpressionsState = (List *)
+			ExecPrepareExpr((Expr *) pkinfo->pi_Expressions,
+							estate);
+		/* Check caller has set up context correctly */
+		Assert(GetPerTupleExprContext(estate)->ecxt_scantuple == slot);
+	}
+	partexpr_item = list_head(pkinfo->pi_ExpressionsState);
+
+	for (i = 0; i < pkinfo->pi_NumKeyAttrs; i++)
+	{
+		AttrNumber	keycol = pkinfo->pi_KeyAttrNumbers[i];
+		Datum		pkDatum;
+		bool		isNull;
+
+		if (keycol != 0)
+		{
+			/*
+			 * Plain column; get the value we need directly from the heap
+			 * tuple.
+			 */
+			pkDatum = slot_getattr(slot, keycol, &isNull);
+		}
+		else
+		{
+			/*
+			 * Partition Key expression --- need to evaluate it.
+			 */
+			if (partexpr_item == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+			pkDatum = ExecEvalExprSwitchContext((ExprState *) lfirst(partexpr_item),
+											   GetPerTupleExprContext(estate),
+											   &isNull,
+											   NULL);
+			partexpr_item = lnext(partexpr_item);
+		}
+		values[i] = pkDatum;
+		isnull[i] = isNull;
+	}
+
+	if (partexpr_item != NULL)
+		elog(ERROR, "wrong number of partition key expressions");
+}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index e59b163..787475f 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3445,6 +3445,14 @@ reindex_relation(Oid relid, int flags, int options)
 	 */
 	rel = heap_open(relid, ShareLock);
 
+	/*
+	 * XXX - Reindexing a partitioned table not supported yet
+	 */
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+					(errmsg("cannot reindex partitioned table \"%s\"", RelationGetRelationName(rel)),
+					 errhint("Run REINDEX on individual partitions one-by-one.")));
+
 	toast_relid = rel->rd_rel->reltoastrelid;
 
 	/*
diff --git a/src/backend/catalog/objectaddress.c b/src/backend/catalog/objectaddress.c
index 052aab1..2a78818 100644
--- a/src/backend/catalog/objectaddress.c
+++ b/src/backend/catalog/objectaddress.c
@@ -1152,7 +1152,8 @@ get_relation_by_qualified_name(ObjectType objtype, List *objname,
 								RelationGetRelationName(relation))));
 			break;
 		case OBJECT_TABLE:
-			if (relation->rd_rel->relkind != RELKIND_RELATION)
+			if (relation->rd_rel->relkind != RELKIND_RELATION &&
+				relation->rd_rel->relkind != RELKIND_PARTITIONED_REL)
 				ereport(ERROR,
 						(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 						 errmsg("\"%s\" is not a table",
@@ -3181,7 +3182,15 @@ getRelationDescription(StringInfo buffer, Oid relid)
 	switch (relForm->relkind)
 	{
 		case RELKIND_RELATION:
-			appendStringInfo(buffer, _("table %s"),
+			if (relForm->relispartition)
+				appendStringInfo(buffer, _("partition %s"),
+					relname, get_rel_name(get_partition_parent(relid, false)));
+			else
+				appendStringInfo(buffer, _("table %s"),
+								 relname);
+			break;
+		case RELKIND_PARTITIONED_REL:
+			appendStringInfo(buffer, _("partitioned table %s"),
 							 relname);
 			break;
 		case RELKIND_INDEX:
@@ -3636,6 +3645,9 @@ getRelationTypeDescription(StringInfo buffer, Oid relid, int32 objectSubId)
 		case RELKIND_RELATION:
 			appendStringInfoString(buffer, "table");
 			break;
+		case RELKIND_PARTITIONED_REL:
+			appendStringInfoString(buffer, "partitioned table");
+			break;
 		case RELKIND_INDEX:
 			appendStringInfoString(buffer, "index");
 			break;
diff --git a/src/backend/catalog/pg_partition.c b/src/backend/catalog/pg_partition.c
new file mode 100644
index 0000000..608299a
--- /dev/null
+++ b/src/backend/catalog/pg_partition.c
@@ -0,0 +1,425 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_partition.c
+ *	  Routines to support manipulation of the pg_partition relation.
+ *
+ * Note: currently, this module only contains inquiry functions; the actual
+ * creation and deletion of pg_partition entries is done in heap.c. Perhaps
+ * someday that code should be moved here.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *	  src/backend/catalog/pg_partition.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/genam.h"
+#include "access/heapam.h"
+#include "access/htup_details.h"
+#include "catalog/heap.h"
+#include "catalog/indexing.h"
+#include "catalog/pg_collation.h"
+#include "catalog/pg_partition.h"
+#include "catalog/pg_partition_fn.h"
+#include "catalog/pg_type.h"
+#include "nodes/nodeFuncs.h"
+#include "nodes/execnodes.h"
+#include "parser/parse_type.h"
+#include "storage/lmgr.h"
+#include "utils/array.h"
+#include "utils/fmgroids.h"
+#include "utils/syscache.h"
+#include "utils/lsyscache.h"
+#include "utils/partition.h"
+#include "utils/tqual.h"
+#include "utils/typcache.h"
+
+static int	oid_cmp(const void *p1, const void *p2);
+
+/*
+ * get_partition_parent
+ *
+ * Returns OID of the parent of partid.
+ */
+Oid
+get_partition_parent(Oid relid, bool recurse)
+{
+	HeapTuple			tuple;
+	Form_pg_partition	form;
+
+	tuple = SearchSysCache1(PARTITIONID, ObjectIdGetDatum(relid));
+
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for partition entry of relation %u", relid);
+
+	form  = (Form_pg_partition) GETSTRUCT(tuple);
+	Assert(form->partparent != InvalidOid);
+
+	if (recurse && is_partition(form->partparent))
+		return get_partition_parent(form->partparent, true);
+
+	ReleaseSysCache(tuple);
+	return form->partparent;
+}
+
+/*
+ * IsPartition - is rel a partition?
+ */
+bool
+IsPartition(Relation rel)
+{
+	return rel->rd_rel->relispartition;
+}
+
+/*
+ * is_partition - is this a partition?
+ *
+ * Checks if the relation with OID 'relid' is a partition based
+ * on its relispartition flag.
+ *
+ * Doesn't really look like this belongs here.
+ */
+bool
+is_partition(Oid relid)
+{
+	HeapTuple	tuple;
+	bool		ispartition;
+
+	tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u", relid);
+
+	ispartition = ((Form_pg_class) GETSTRUCT(tuple))->relispartition;
+	ReleaseSysCache(tuple);
+
+	return ispartition;
+}
+
+/*
+ * IsPartitionedRel - is rel partitioned?
+ */
+bool
+IsPartitionedRel(Relation rel)
+{
+	return rel->rd_rel->relkind == RELKIND_PARTITIONED_REL ? true : false;
+}
+
+/*
+ * is_partitioned - is relation with OID relid partitioned?
+ *
+ * Doesn't really look like this belongs here.
+ */
+bool
+is_partitioned(Oid relid)
+{
+	HeapTuple	tuple;
+	char		relkind;
+
+	tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(relid));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u", relid);
+
+	relkind = ((Form_pg_class) GETSTRUCT(tuple))->relkind;
+	ReleaseSysCache(tuple);
+
+	return relkind == RELKIND_PARTITIONED_REL ? true : false;
+}
+
+/*
+ * find_partitions
+ *
+ * Returns a list of direct partitions of partitioned table with OID 'relid'.
+ * Found partitions are locked in an order that follows the sorted order of
+ * their OIDs.
+ */
+List *
+find_partitions(Oid relid, LOCKMODE lockmode)
+{
+	List		   *list = NIL;
+	Relation		partitionRel;
+	SysScanDesc		scan;
+	ScanKeyData		skey[1];
+	HeapTuple		tuple;
+	Oid				partrelid;
+	Oid			   *oidarr;
+	int				maxoids,
+					numoids,
+					i;
+
+	/* Fairly certain this holds if we got here at all. */
+	Assert(is_partitioned(relid));
+
+	/* Scan pg_partition and build a working array of partition OIDs. */
+	maxoids = 32;
+	oidarr = (Oid *) palloc(maxoids * sizeof(Oid));
+	numoids = 0;
+
+	partitionRel = heap_open(PartitionRelationId, AccessShareLock);
+	ScanKeyInit(&skey[0],
+				Anum_pg_partition_partparent,
+				BTEqualStrategyNumber, F_OIDEQ,
+				relid);
+
+	scan = systable_beginscan(partitionRel, PartitionParentIndexId, true,
+								NULL, 1, skey);
+
+	while ((tuple = systable_getnext(scan)) != NULL)
+	{
+		partrelid = ((Form_pg_partition) GETSTRUCT(tuple))->partitionid;
+		if (numoids >= maxoids)
+		{
+			maxoids *= 2;
+			oidarr = (Oid *) repalloc(oidarr, maxoids * sizeof(Oid));
+		}
+		oidarr[numoids++] = partrelid;
+	}
+
+	systable_endscan(scan);
+
+	heap_close(partitionRel, AccessShareLock);
+
+	/*
+	 * If we found more than one partition, sort them by OID. This is important
+	 * since we need to be sure all backends lock partitions in the same order
+	 * to avoid needless deadlocks.
+	 */
+	if (numoids > 1)
+		qsort(oidarr, numoids, sizeof(Oid), oid_cmp);
+
+	/* Acquire locks and build the result list. */
+	for (i = 0; i < numoids; i++)
+	{
+		partrelid = oidarr[i];
+
+		if (lockmode != NoLock)
+		{
+			/* Get the lock to synchronize against concurrent drop */
+			LockRelationOid(partrelid, lockmode);
+
+			/*
+			 * Now that we have the lock, double-check to see if the relation
+			 * really exists or not.  If not, assume it was dropped while we
+			 * waited to acquire lock, and ignore it.
+			 */
+			if (!SearchSysCacheExists1(RELOID, ObjectIdGetDatum(partrelid)))
+			{
+				/* Release useless lock */
+				UnlockRelationOid(partrelid, lockmode);
+				/* And ignore this relation */
+				continue;
+			}
+		}
+
+		list = lappend_oid(list, partrelid);
+	}
+
+	pfree(oidarr);
+
+	return list;
+}
+
+/*
+ * find_all_partitions
+ *
+ * Finds all partitions of partitioned table with OID 'relid' including
+ * those of its partitions (if they are partitioned further).
+ */
+List *
+find_all_partitions(Oid relid, LOCKMODE lockmode)
+{
+	List	   *all_partitions = find_partitions(relid, lockmode);
+	ListCell   *cell;
+
+	foreach (cell, all_partitions)
+	{
+		Oid			relid = lfirst_oid(cell);
+		List	   *partitions;
+		ListCell   *innercell;
+
+		/*
+		 * If this relation itself is partitioned, queue its partitions
+		 * for consideration.
+		 */
+		if (is_partitioned(relid))
+		{
+			partitions = find_partitions(relid, lockmode);
+
+			foreach(innercell, partitions)
+				all_partitions = lappend_oid(all_partitions,
+													lfirst_oid(innercell));
+		}
+	}
+
+	return all_partitions;
+}
+
+/*
+ * free_partitions -
+ *
+ * pfree's the partitions in p that were created by GetPartitionBounds
+ */
+void
+free_partitions(PartitionBoundInfo **p, int count)
+{
+	int	i;
+
+	for (i = 0; i < count; i++)
+	{
+		/*
+		 * Ugly hack to delete only those that GetPartitionBounds() had found.
+		 */
+		if (p[i]->oid != InvalidOid)
+		{
+			if (p[i]->rangemins)
+				pfree(p[i]->rangemins);
+			if (p[i]->rangemaxs)
+				pfree(p[i]->rangemaxs);
+			if (p[i]->listvalues)
+				pfree(p[i]->listvalues);
+
+			pfree(p[i]);
+		}
+	}
+	pfree(p);
+}
+
+/*
+ * GetPartitionBounds
+ *
+ * Returns a list of immediate partitions of rel including OIDs
+ * and bound info.
+ */
+PartitionBoundInfo **
+GetPartitionBounds(Relation rel, int *numparts)
+{
+	List	*partoids;
+	PartitionBoundInfo **partitions = NULL;
+
+	partoids = find_partitions(RelationGetRelid(rel), NoLock);
+	*numparts = list_length(partoids);
+
+	/* Copy info into the array of Partition to return. */
+	if (*numparts)
+	{
+		int			i;
+		int			partnatts = rel->rd_pkey->partnatts;
+		ListCell   *cell = NULL;
+		PartitionKeyTypeInfo *typinfo = get_key_type_info(rel);
+
+		partitions = (PartitionBoundInfo **) palloc0(*numparts
+											* sizeof(PartitionBoundInfo *));
+
+		i = 0;
+		foreach(cell, partoids)
+		{
+			HeapTuple			tuple;
+			Form_pg_partition	form;
+			int					j;
+			int					rangenbounds;
+			Datum			   *rangebounds;
+			Datum				datum;
+			bool				isnull;
+			Oid 				partitionid;
+			PartitionBoundInfo *result = NULL;
+
+			partitionid = lfirst_oid(cell);
+
+			tuple = SearchSysCache1(PARTITIONID, partitionid);
+
+			/* If no tuple found, it means the entry was just dropped. */
+			if (!HeapTupleIsValid(tuple))
+				elog(ERROR, "cache lookup failed for partition %u",
+						partitionid);
+
+			result = (PartitionBoundInfo *) palloc0(sizeof(PartitionBoundInfo));
+			result->oid = partitionid;
+			result->partnatts = partnatts;
+
+			form = (Form_pg_partition) GETSTRUCT(tuple);
+
+			datum = SysCacheGetAttr(PARTITIONID, tuple,
+									Anum_pg_partition_partlistvalues, &isnull);
+			if (!isnull)
+			{
+
+				deconstruct_array(DatumGetArrayTypeP(datum),
+								typinfo->typid[0],
+								typinfo->typlen[0],
+								typinfo->typbyval[0],
+								typinfo->typalign[0],
+								&result->listvalues, NULL, &result->listnvalues);
+
+				ReleaseSysCache(tuple);
+				partitions[i++] = result;
+				continue;
+			}
+
+			/* A range partition. */
+			datum = SysCacheGetAttr(PARTITIONID, tuple,
+									Anum_pg_partition_partrangebounds, &isnull);
+
+			/* This better not be NULL. */
+			Assert(!isnull);
+
+			/* There is a {min, max} array for every partition column. */
+			deconstruct_array(DatumGetArrayTypeP(datum),
+								ANYARRAYOID, -1, false, 'd',
+								&rangebounds, NULL, &rangenbounds);
+			/* Paranoia. */
+			Assert(rangenbounds = result->partnatts);
+
+			result->rangemins = (Datum *) palloc0(partnatts * sizeof(Datum));
+			result->rangemaxs = (Datum *) palloc0(partnatts * sizeof(Datum));
+
+			for (j = 0; j < result->partnatts; j++)
+			{
+				ArrayType  *arr = DatumGetArrayTypeP(rangebounds[j]);
+				Datum	   *datum;
+				bool	   *nulls;
+				int			dummy;
+
+				deconstruct_array(arr, typinfo->typid[j],
+									typinfo->typlen[j],
+									typinfo->typbyval[j],
+									typinfo->typalign[j],
+									&datum, &nulls, &dummy);
+
+				result->rangeminnull = nulls[0];
+				result->rangemaxnull = nulls[1];
+
+				if (!result->rangeminnull)
+					result->rangemins[j] = datum[0];
+
+				if (!result->rangemaxnull)
+					result->rangemaxs[j] = datum[1];
+			}
+
+			ReleaseSysCache(tuple);
+
+			partitions[i++] = result;
+		}
+
+		pfree(typinfo);
+	}
+
+	return partitions;
+}
+
+/* The qsort comparison function. */
+static int
+oid_cmp(const void *p1, const void *p2)
+{
+	Oid			v1 = *((const Oid *) p1);
+	Oid			v2 = *((const Oid *) p2);
+
+	if (v1 < v2)
+		return -1;
+	if (v1 > v2)
+		return 1;
+	return 0;
+}
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index 861048f..b9c54c9 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -196,6 +196,22 @@ analyze_rel(Oid relid, RangeVar *relation, int options,
 	}
 
 	/*
+	 * XXX - Analyzing a partitioned table not supported yet
+	 */
+	if (onerel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+	{
+		if(!IsAutoVacuumWorkerProcess())
+			ereport(ERROR,
+					(errmsg("cannot analyze partitioned table \"%s\"", RelationGetRelationName(onerel)),
+					 errdetail("ANALYZE on partitioned tables is not implemented yet."),
+					 errhint("ANALYZE individual partitions separately.")));
+		else
+			ereport(WARNING,
+					(errmsg("skipping \"%s\" --- cannot analyze partitioned tables",
+							RelationGetRelationName(onerel))));
+	}
+
+	/*
 	 * Check that it's a plain table, materialized view, or foreign table; we
 	 * used to do this in get_rel_oids() but seems safer to check after we've
 	 * locked the relation.
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index 7ab4874..94d9e64 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -126,6 +126,11 @@ cluster(ClusterStmt *stmt, bool isTopLevel)
 					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 			   errmsg("cannot cluster temporary tables of other sessions")));
 
+		if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+			ereport(ERROR,
+					(errmsg("cannot cluster partitioned table \"%s\"", RelationGetRelationName(rel)),
+					 errhint("Run CLUSTER on individual partitions one-by-one.")));
+
 		if (stmt->indexname == NULL)
 		{
 			ListCell   *index;
diff --git a/src/backend/commands/comment.c b/src/backend/commands/comment.c
index 6d8c006..3a5e8da 100644
--- a/src/backend/commands/comment.c
+++ b/src/backend/commands/comment.c
@@ -92,6 +92,7 @@ CommentObject(CommentStmt *stmt)
 			 * failures.
 			 */
 			if (relation->rd_rel->relkind != RELKIND_RELATION &&
+				relation->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 				relation->rd_rel->relkind != RELKIND_VIEW &&
 				relation->rd_rel->relkind != RELKIND_MATVIEW &&
 				relation->rd_rel->relkind != RELKIND_COMPOSITE_TYPE &&
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 8db1b35..8888566 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -1655,6 +1655,13 @@ BeginCopyTo(Relation rel,
 
 	if (rel != NULL && rel->rd_rel->relkind != RELKIND_RELATION)
 	{
+		/* a partitioned table itself does not support heap scan */
+		if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot copy from partitioned table \"%s\"",
+							RelationGetRelationName(rel)),
+					 errhint("Try the COPY (SELECT ...) TO variant.")));
 		if (rel->rd_rel->relkind == RELKIND_VIEW)
 			ereport(ERROR,
 					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
@@ -2170,6 +2177,7 @@ CopyFrom(CopyState cstate)
 	Datum	   *values;
 	bool	   *nulls;
 	ResultRelInfo *resultRelInfo;
+	ResultRelInfo *saved_resultRelInfo = NULL;	/* for partitioned table case */
 	EState	   *estate = CreateExecutorState(); /* for ExecConstraints() */
 	ExprContext *econtext;
 	TupleTableSlot *myslot;
@@ -2178,7 +2186,7 @@ CopyFrom(CopyState cstate)
 	ErrorContextCallback errcallback;
 	CommandId	mycid = GetCurrentCommandId(true);
 	int			hi_options = 0; /* start with default heap_insert options */
-	BulkInsertState bistate;
+	BulkInsertState bistate = NULL;
 	uint64		processed = 0;
 	bool		useHeapMultiInsert;
 	int			nBufferedTuples = 0;
@@ -2190,7 +2198,8 @@ CopyFrom(CopyState cstate)
 
 	Assert(cstate->rel);
 
-	if (cstate->rel->rd_rel->relkind != RELKIND_RELATION)
+	if (cstate->rel->rd_rel->relkind != RELKIND_RELATION &&
+			cstate->rel->rd_rel->relkind != RELKIND_PARTITIONED_REL)
 	{
 		if (cstate->rel->rd_rel->relkind == RELKIND_VIEW)
 			ereport(ERROR,
@@ -2219,6 +2228,13 @@ CopyFrom(CopyState cstate)
 							RelationGetRelationName(cstate->rel))));
 	}
 
+	/* prevent copying to partitions directly */
+	if (cstate->rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot copy to a partition of another table"),
+				 errhint("Perform COPY on the parent instead.")));
+
 	tupDesc = RelationGetDescr(cstate->rel);
 
 	/*----------
@@ -2321,10 +2337,14 @@ CopyFrom(CopyState cstate)
 	 * expressions. Such triggers or expressions might query the table we're
 	 * inserting to, and act differently if the tuples that have already been
 	 * processed and prepared for insertion are not there.
+	 *
+	 * Also, Do not use multi- mode if inserting into a partitioned table
+	 * because we need to determine a partition for each row individually.
 	 */
 	if ((resultRelInfo->ri_TrigDesc != NULL &&
 		 (resultRelInfo->ri_TrigDesc->trig_insert_before_row ||
 		  resultRelInfo->ri_TrigDesc->trig_insert_instead_row)) ||
+		resultRelInfo->ri_Partitions != NULL ||
 		cstate->volatile_defexprs)
 	{
 		useHeapMultiInsert = false;
@@ -2349,7 +2369,10 @@ CopyFrom(CopyState cstate)
 	values = (Datum *) palloc(tupDesc->natts * sizeof(Datum));
 	nulls = (bool *) palloc(tupDesc->natts * sizeof(bool));
 
-	bistate = GetBulkInsertState();
+	/* bistate confuses heap_insert() as to which relation's buffers to use */
+	if (!resultRelInfo->ri_Partitions)
+		bistate = GetBulkInsertState();
+
 	econtext = GetPerTupleExprContext(estate);
 
 	/* Set up callback to identify error line number */
@@ -2421,6 +2444,18 @@ CopyFrom(CopyState cstate)
 			if (cstate->rel->rd_att->constr)
 				ExecConstraints(resultRelInfo, slot, estate);
 
+			/*
+			 * Switch resultRelInfo to one corresponding to the partition
+			 * that this tuple maps to.
+			 */
+			if (resultRelInfo->ri_Partitions)
+			{
+				saved_resultRelInfo = resultRelInfo;
+				econtext->ecxt_scantuple = slot;	/* for partition key expressions */
+				resultRelInfo = ExecFindPartition(resultRelInfo, slot, estate);
+				estate->es_result_relation_info = resultRelInfo;
+			}
+
 			if (useHeapMultiInsert)
 			{
 				/* Add this tuple to the tuple buffer */
@@ -2451,7 +2486,8 @@ CopyFrom(CopyState cstate)
 				List	   *recheckIndexes = NIL;
 
 				/* OK, store the tuple and create index entries for it */
-				heap_insert(cstate->rel, tuple, mycid, hi_options, bistate);
+				heap_insert(resultRelInfo->ri_RelationDesc,
+									tuple, mycid, hi_options, bistate);
 
 				if (resultRelInfo->ri_NumIndices > 0)
 					recheckIndexes = ExecInsertIndexTuples(slot, &(tuple->t_self),
@@ -2472,6 +2508,17 @@ CopyFrom(CopyState cstate)
 			 */
 			processed++;
 		}
+
+		/* Restore the resultRelInfo, if had been switched. */
+		if (saved_resultRelInfo)
+		{
+			heap_close(resultRelInfo->ri_RelationDesc, RowExclusiveLock);
+			ExecCloseIndices(resultRelInfo);
+			pfree(resultRelInfo);
+			resultRelInfo = saved_resultRelInfo;
+			estate->es_result_relation_info = resultRelInfo;
+			saved_resultRelInfo = NULL; /* aka, paranoia! */
+		}
 	}
 
 	/* Flush any remaining buffered tuples */
@@ -2484,7 +2531,8 @@ CopyFrom(CopyState cstate)
 	/* Done, clean up */
 	error_context_stack = errcallback.previous;
 
-	FreeBulkInsertState(bistate);
+	if (bistate)
+		FreeBulkInsertState(bistate);
 
 	MemoryContextSwitchTo(oldcontext);
 
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index b450bcf..d376bda 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -65,8 +65,6 @@ static void ComputeIndexAttrs(IndexInfo *indexInfo,
 				  char *accessMethodName, Oid accessMethodId,
 				  bool amcanorder,
 				  bool isconstraint);
-static Oid GetIndexOpClass(List *opclass, Oid attrType,
-				char *accessMethodName, Oid accessMethodId);
 static char *ChooseIndexName(const char *tabname, Oid namespaceId,
 				List *colnames, List *exclusionOpNames,
 				bool primary, bool isconstraint);
@@ -375,6 +373,11 @@ DefineIndex(Oid relationId,
 					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 					 errmsg("cannot create index on foreign table \"%s\"",
 							RelationGetRelationName(rel))));
+		else if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot create index on partitioned table \"%s\"",
+							RelationGetRelationName(rel))));
 		else
 			ereport(ERROR,
 					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
@@ -1211,9 +1214,9 @@ ComputeIndexAttrs(IndexInfo *indexInfo,
 /*
  * Resolve possibly-defaulted operator class specification
  */
-static Oid
+Oid
 GetIndexOpClass(List *opclass, Oid attrType,
-				char *accessMethodName, Oid accessMethodId)
+				const char *accessMethodName, Oid accessMethodId)
 {
 	char	   *schemaname;
 	char	   *opcname;
diff --git a/src/backend/commands/lockcmds.c b/src/backend/commands/lockcmds.c
index a167082..3897b78 100644
--- a/src/backend/commands/lockcmds.c
+++ b/src/backend/commands/lockcmds.c
@@ -88,7 +88,7 @@ RangeVarCallbackForLockTable(const RangeVar *rv, Oid relid, Oid oldrelid,
 								 * check */
 
 	/* Currently, we only allow plain tables to be locked */
-	if (relkind != RELKIND_RELATION)
+	if (relkind != RELKIND_RELATION && relkind != RELKIND_PARTITIONED_REL)
 		ereport(ERROR,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("\"%s\" is not a table",
diff --git a/src/backend/commands/policy.c b/src/backend/commands/policy.c
index bcf4a8f..e239b43 100644
--- a/src/backend/commands/policy.c
+++ b/src/backend/commands/policy.c
@@ -88,7 +88,7 @@ RangeVarCallbackForPolicy(const RangeVar *rv, Oid relid, Oid oldrelid,
 						rv->relname)));
 
 	/* Relation type MUST be a table. */
-	if (relkind != RELKIND_RELATION)
+	if (relkind != RELKIND_RELATION && relkind != RELKIND_PARTITIONED_REL)
 		ereport(ERROR,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("\"%s\" is not a table", rv->relname)));
diff --git a/src/backend/commands/seclabel.c b/src/backend/commands/seclabel.c
index 1ef98ce..8df2e23 100644
--- a/src/backend/commands/seclabel.c
+++ b/src/backend/commands/seclabel.c
@@ -107,6 +107,7 @@ ExecSecLabelStmt(SecLabelStmt *stmt)
 			 * are the only relkinds for which pg_dump will dump labels).
 			 */
 			if (relation->rd_rel->relkind != RELKIND_RELATION &&
+				relation->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 				relation->rd_rel->relkind != RELKIND_VIEW &&
 				relation->rd_rel->relkind != RELKIND_MATVIEW &&
 				relation->rd_rel->relkind != RELKIND_COMPOSITE_TYPE &&
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 126b119..9036075 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -37,6 +37,9 @@
 #include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_namespace.h"
 #include "catalog/pg_opclass.h"
+#include "catalog/pg_partition.h"
+#include "catalog/pg_partition_fn.h"
+#include "catalog/pg_partitioned_rel.h"
 #include "catalog/pg_tablespace.h"
 #include "catalog/pg_trigger.h"
 #include "catalog/pg_type.h"
@@ -82,10 +85,12 @@
 #include "storage/smgr.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
+#include "utils/datum.h"
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partition.h"
 #include "utils/relcache.h"
 #include "utils/ruleutils.h"
 #include "utils/snapmgr.h"
@@ -259,6 +264,12 @@ struct DropRelationCallbackState
 	bool		concurrent;
 };
 
+/* for find_att_reference_walker */
+typedef struct
+{
+	AttrNumber	attnum;
+} find_att_reference_context;
+
 /* Alter table target-type flags for ATSimplePermissions */
 #define		ATT_TABLE				0x0001
 #define		ATT_VIEW				0x0002
@@ -322,6 +333,8 @@ static void ATSimpleRecursion(List **wqueue, Relation rel,
 				  AlterTableCmd *cmd, bool recurse, LOCKMODE lockmode);
 static void ATTypedTableRecursion(List **wqueue, Relation rel, AlterTableCmd *cmd,
 					  LOCKMODE lockmode);
+static void ATPartitionedTableRecursion(List **wqueue, Relation rel, AlterTableCmd *cmd,
+								LOCKMODE lockmode);
 static List *find_typed_table_dependencies(Oid typeOid, const char *typeName,
 							  DropBehavior behavior);
 static void ATPrepAddColumn(List **wqueue, Relation rel, bool recurse, bool recursing,
@@ -398,7 +411,7 @@ static void change_owner_recurse_to_sequences(Oid relationOid,
 static ObjectAddress ATExecClusterOn(Relation rel, const char *indexName,
 				LOCKMODE lockmode);
 static void ATExecDropCluster(Relation rel, LOCKMODE lockmode);
-static bool ATPrepChangePersistence(Relation rel, bool toLogged);
+static bool ATPrepChangePersistence(Relation rel, bool toLogged, bool recursing);
 static void ATPrepSetTableSpace(AlteredTableInfo *tab, Relation rel,
 					char *tablespacename, LOCKMODE lockmode);
 static void ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode);
@@ -419,6 +432,8 @@ static void ATExecReplicaIdentity(Relation rel, ReplicaIdentityStmt *stmt, LOCKM
 static void ATExecGenericOptions(Relation rel, List *options);
 static void ATExecEnableRowSecurity(Relation rel);
 static void ATExecDisableRowSecurity(Relation rel);
+static void ATExecAttachPartition(Relation parent, PartitionDef *partition);
+static void ATExecDetachPartition(Relation parent, PartitionDef *partition);
 
 static void copy_relation_data(SMgrRelation rel, SMgrRelation dst,
 				   ForkNumber forkNum, char relpersistence);
@@ -428,6 +443,28 @@ static void RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid,
 								Oid oldRelOid, void *arg);
 static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid,
 								 Oid oldrelid, void *arg);
+static PartitionBy *transformPartitionBy(Relation rel, PartitionBy *partitionby);
+static PartitionKeyInfo *ComputePartitionAttrs(PartitionKeyInfo *pkinfo,
+								Oid *partClassOids, Oid relid, List *partParams);
+static PartitionBoundInfo *EvalPartitionBound(Relation parent,
+								PartitionDef *partdef, int *listnvalues);
+static Datum *evalPartitionListBound(List *values, PartitionKeyTypeInfo *typinfo,
+								int nvalues);
+static Datum *evalPartitionRangeBound(List *values, PartitionKeyTypeInfo *typinfo,
+								int partnatts);
+static void CheckPartitionAgainstExisting(Relation parent, PartitionBoundInfo *new,
+								Datum **final);
+static void ATPrepColumnDefault(Relation rel, bool recursing);
+static void ATPrepDropNotNull(Relation rel, bool recursing);
+static void ATPrepSetNotNull(Relation rel, bool recursing);
+static void ATPrepAddConstraint(Relation rel);
+static void ATPrepDropConstraint(Relation rel);
+static void ATPrepAlterConstraint(Relation rel);
+static void ATExecChangePersistence(Relation rel, char persistence);
+static void SetRelationPersistence(Oid relationId, char persistence);
+static void remove_partitioning_dependency(Oid partitionOid, Oid parentOid);
+static Oid RangeVarGetTablespace(RangeVar *relation);
+static void SetRelationSpcFileNode(Relation rel, Oid newTableSpace, Oid newFileNode);
 
 
 /* ----------------------------------------------------------------
@@ -471,6 +508,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 	static char *validnsps[] = HEAP_RELOPT_NAMESPACES;
 	Oid			ofTypeId;
 	ObjectAddress address;
+	PartitionKeyInfo *pkinfo = NULL;
+	Oid		   partOpClassOids[PARTITION_MAX_KEYS];
 
 	/*
 	 * Truncate relname to appropriate length (probably a waste of time, as
@@ -515,6 +554,11 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 	{
 		tablespaceId = get_tablespace_oid(stmt->tablespacename, false);
 	}
+	else if (stmt->partitionOf)
+	{
+		tablespaceId = RangeVarGetTablespace(stmt->partitionOf);
+		/* note InvalidOid is OK in this case */
+	}
 	else
 	{
 		tablespaceId = GetDefaultTablespace(stmt->relation->relpersistence);
@@ -705,6 +749,48 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 		AddRelationNewConstraints(rel, rawDefaults, stmt->constraints,
 								  true, true, false);
 
+	/*
+	 * Store the partition key into pg_partitioned_rel. It consists of columns
+	 * constituting the key and optional opclass to use with it along with the
+	 * partitioning strategy (range or list).
+	 */
+	if (stmt->partitionby)
+	{
+		int	numPartitionAttrs;
+
+		Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_REL);
+
+		/* sanity */
+		numPartitionAttrs = list_length(stmt->partitionby->partParams);
+		if (numPartitionAttrs < 1)
+			ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("must specify at least one column in partition key")));
+		if (numPartitionAttrs > PARTITION_MAX_KEYS)
+			ereport(ERROR,
+				(errcode(ERRCODE_TOO_MANY_COLUMNS),
+				 errmsg("cannot use more than %d columns in partition key",
+				 INDEX_MAX_KEYS)));
+		if (stmt->partitionby->strategy == PARTITION_STRAT_LIST
+											&& numPartitionAttrs > 1)
+			ereport(ERROR,
+				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+				 errmsg("cannot use more than one column in partition key"),
+				 errdetail("Only one column allowed when using the list strategy.")));
+
+		/*
+		 * Take care of any expressions in the partition key. This should ideally
+		 * have been done in parse_utilcmd.c but could not be done in this case
+		 * because we need "rel" to be able to look at the key columns.
+		 */
+		stmt->partitionby = transformPartitionBy(rel, stmt->partitionby);
+
+		pkinfo = ComputePartitionAttrs(pkinfo, partOpClassOids, relationId,
+										stmt->partitionby->partParams);
+
+		StorePartitionKey(rel, pkinfo, partOpClassOids, stmt->partitionby->strategy);
+	}
+
 	ObjectAddressSet(address, RelationRelationId, relationId);
 
 	/*
@@ -950,7 +1036,25 @@ RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid, Oid oldRelOid,
 		return;					/* concurrently dropped, so nothing to do */
 	classform = (Form_pg_class) GETSTRUCT(tuple);
 
-	if (classform->relkind != relkind)
+	/*
+	 * Reject dropping tables marked as partitions. They need to be detached
+	 * from their partitioning parent before they can be safely dropped.
+	 */
+	if (classform->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot drop table \"%s\"", rel->relname),
+				 errdetail("\"%s\" is a partition of table \"%s\".", rel->relname,
+					get_rel_name(get_partition_parent(relOid, false))),
+				 errhint("You will need to detach the partition first.")));
+
+	/*
+	 * Both normal and partitioned tables are dropped using DROP TABLE; so
+	 * both relkinds are possible but the caller can only pass one.
+	 */
+	if (classform->relkind != relkind &&
+				(relkind == RELKIND_RELATION &&
+					classform->relkind != RELKIND_PARTITIONED_REL))
 		DropErrorMsgWrongType(rel->relname, classform->relkind, relkind);
 
 	/* Allow DROP to either table owner or schema owner */
@@ -1516,6 +1620,18 @@ MergeAttributes(List *schema, List *supers, char relpersistence,
 		 */
 		relation = heap_openrv(parent, ShareUpdateExclusiveLock);
 
+		/* special case partitioned tables and partitions */
+		if (relation->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot inherit from partitioned table \"%s\"",
+							parent->relname)));
+		if (relation->rd_rel->relispartition)
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot inherit from partition \"%s\"",
+							parent->relname)));
+
 		if (relation->rd_rel->relkind != RELKIND_RELATION &&
 			relation->rd_rel->relkind != RELKIND_FOREIGN_TABLE)
 			ereport(ERROR,
@@ -2153,6 +2269,7 @@ renameatt_check(Oid myrelid, Form_pg_class classform, bool recursing)
 	 * restriction.
 	 */
 	if (relkind != RELKIND_RELATION &&
+		relkind != RELKIND_PARTITIONED_REL &&
 		relkind != RELKIND_VIEW &&
 		relkind != RELKIND_MATVIEW &&
 		relkind != RELKIND_COMPOSITE_TYPE &&
@@ -3046,6 +3163,15 @@ AlterTableGetLockLevel(List *cmds)
 				cmd_lockmode = AlterTableGetRelOptionsLockLevel((List *) cmd->def);
 				break;
 
+				/*
+				 * The following lock refers to one taken on the parent for
+				 * this ALTER TABLE invocation.
+				 */
+			case AT_AttachPartition:
+			case AT_DetachPartition:
+				cmd_lockmode = AccessExclusiveLock;
+				break;
+
 			default:			/* oops */
 				elog(ERROR, "unrecognized alter table type: %d",
 					 (int) cmd->subtype);
@@ -3121,14 +3247,16 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 	cmd = copyObject(cmd);
 
 	/*
-	 * Do permissions checking, recursion to child tables if needed, and any
-	 * additional phase-1 processing needed.
+	 * Do permissions checking, recursion to child tables or partitions if
+	 * needed, and any additional phase-1 processing needed.
 	 */
 	switch (cmd->subtype)
 	{
 		case AT_AddColumn:		/* ADD COLUMN */
 			ATSimplePermissions(rel,
 						 ATT_TABLE | ATT_COMPOSITE_TYPE | ATT_FOREIGN_TABLE);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			ATPrepAddColumn(wqueue, rel, recurse, recursing, false, cmd,
 							lockmode);
 			/* Recursion occurs during execution phase */
@@ -3152,23 +3280,31 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			 */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_VIEW | ATT_FOREIGN_TABLE);
 			ATSimpleRecursion(wqueue, rel, cmd, recurse, lockmode);
-			/* No command-specific prep needed */
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
+			ATPrepColumnDefault(rel, recursing);
 			pass = cmd->def ? AT_PASS_ADD_CONSTR : AT_PASS_DROP;
 			break;
 		case AT_DropNotNull:	/* ALTER COLUMN DROP NOT NULL */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
 			ATSimpleRecursion(wqueue, rel, cmd, recurse, lockmode);
-			/* No command-specific prep needed */
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
+			ATPrepDropNotNull(rel, recursing);
 			pass = AT_PASS_DROP;
 			break;
 		case AT_SetNotNull:		/* ALTER COLUMN SET NOT NULL */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
 			ATSimpleRecursion(wqueue, rel, cmd, recurse, lockmode);
-			/* No command-specific prep needed */
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
+			ATPrepSetNotNull(rel, recursing);
 			pass = AT_PASS_ADD_CONSTR;
 			break;
 		case AT_SetStatistics:	/* ALTER COLUMN SET STATISTICS */
 			ATSimpleRecursion(wqueue, rel, cmd, recurse, lockmode);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			/* Performs own permission checks */
 			ATPrepSetStatistics(rel, cmd->name, cmd->def, lockmode);
 			pass = AT_PASS_MISC;
@@ -3188,6 +3324,8 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 		case AT_DropColumn:		/* DROP COLUMN */
 			ATSimplePermissions(rel,
 						 ATT_TABLE | ATT_COMPOSITE_TYPE | ATT_FOREIGN_TABLE);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			ATPrepDropColumn(wqueue, rel, recurse, recursing, cmd, lockmode);
 			/* Recursion occurs during execution phase */
 			pass = AT_PASS_DROP;
@@ -3200,8 +3338,10 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_AddConstraint:	/* ADD CONSTRAINT */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
-			/* Recursion occurs during execution phase */
-			/* No command-specific prep needed except saving recurse flag */
+			/* The partitioned table recursion does not occur at all */
+			/* Simple recursion occurs during execution phase */
+			ATPrepAddConstraint(rel);
+			/* Save recurse flag */
 			if (recurse)
 				cmd->subtype = AT_AddConstraintRecurse;
 			pass = AT_PASS_ADD_CONSTR;
@@ -3214,8 +3354,10 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_DropConstraint:	/* DROP CONSTRAINT */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
-			/* Recursion occurs during execution phase */
-			/* No command-specific prep needed except saving recurse flag */
+			/* The partitioned table recursion does not occur at all */
+			/* Simple recursion occurs during execution phase */
+			ATPrepDropConstraint(rel);
+			/* Save recurse flag */
 			if (recurse)
 				cmd->subtype = AT_DropConstraintRecurse;
 			pass = AT_PASS_DROP;
@@ -3223,8 +3365,13 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 		case AT_AlterColumnType:		/* ALTER COLUMN TYPE */
 			ATSimplePermissions(rel,
 						 ATT_TABLE | ATT_COMPOSITE_TYPE | ATT_FOREIGN_TABLE);
-			/* Performs own recursion */
+			/* Performs own (simple) recursion */
 			ATPrepAlterColumnType(wqueue, tab, rel, recurse, recursing, cmd, lockmode);
+			/*
+			 * Take care of partitions of this relation if partitioned
+			 * XXX - unlike other places, after ATPrep* is done.
+			 */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			pass = AT_PASS_ALTER_TYPE;
 			break;
 		case AT_AlterColumnGenericOptions:
@@ -3234,7 +3381,8 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			pass = AT_PASS_MISC;
 			break;
 		case AT_ChangeOwner:	/* ALTER OWNER */
-			/* This command never recurses */
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
@@ -3247,7 +3395,9 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_SetLogged:		/* SET LOGGED */
 			ATSimplePermissions(rel, ATT_TABLE);
-			tab->chgPersistence = ATPrepChangePersistence(rel, true);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
+			tab->chgPersistence = ATPrepChangePersistence(rel, true, recursing);
 			/* force rewrite if necessary; see comment in ATRewriteTables */
 			if (tab->chgPersistence)
 			{
@@ -3258,7 +3408,9 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_SetUnLogged:	/* SET UNLOGGED */
 			ATSimplePermissions(rel, ATT_TABLE);
-			tab->chgPersistence = ATPrepChangePersistence(rel, false);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
+			tab->chgPersistence = ATPrepChangePersistence(rel, false, recursing);
 			/* force rewrite if necessary; see comment in ATRewriteTables */
 			if (tab->chgPersistence)
 			{
@@ -3269,13 +3421,17 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_AddOids:		/* SET WITH OIDS */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			if (!rel->rd_rel->relhasoids || recursing)
 				ATPrepAddOids(wqueue, rel, recurse, cmd, lockmode);
-			/* Recursion occurs during execution phase */
+			/* Simple recursion occurs during execution phase */
 			pass = AT_PASS_ADD_COL;
 			break;
 		case AT_DropOids:		/* SET WITHOUT OIDS */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			/* Performs own recursion */
 			if (rel->rd_rel->relhasoids)
 			{
@@ -3290,6 +3446,8 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_SetTableSpace:	/* SET TABLESPACE */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_MATVIEW | ATT_INDEX);
+			/* Take care of partitions of this relation if partitioned. */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
 			/* This command never recurses */
 			ATPrepSetTableSpace(tab, rel, cmd->name, lockmode);
 			pass = AT_PASS_MISC;	/* doesn't actually matter */
@@ -3316,11 +3474,15 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			break;
 		case AT_AlterConstraint:		/* ALTER CONSTRAINT */
 			ATSimplePermissions(rel, ATT_TABLE);
+			/* The partitioned table recursion does not occur at all */
+			ATPrepAlterConstraint(rel);
 			pass = AT_PASS_MISC;
 			break;
 		case AT_ValidateConstraint:		/* VALIDATE CONSTRAINT */
 			ATSimplePermissions(rel, ATT_TABLE | ATT_FOREIGN_TABLE);
-			/* Recursion occurs during execution phase */
+			/* Take care of partitions of this relation if partitioned */
+			ATPartitionedTableRecursion(wqueue, rel, cmd, lockmode);
+			/* Simple recursion occurs during execution phase */
 			/* No command-specific prep needed except saving recurse flag */
 			if (recurse)
 				cmd->subtype = AT_ValidateConstraintRecurse;
@@ -3330,7 +3492,6 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			ATSimplePermissions(rel, ATT_TABLE | ATT_MATVIEW);
 			pass = AT_PASS_MISC;
 			/* This command never recurses */
-			/* No command-specific prep needed */
 			break;
 		case AT_EnableTrig:		/* ENABLE TRIGGER variants */
 		case AT_EnableAlwaysTrig:
@@ -3353,7 +3514,6 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 		case AT_DisableRowSecurity:
 			ATSimplePermissions(rel, ATT_TABLE);
 			/* These commands never recurse */
-			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
 		case AT_GenericOptions:
@@ -3361,6 +3521,12 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
 			/* No command-specific prep needed */
 			pass = AT_PASS_MISC;
 			break;
+		case AT_AttachPartition:
+		case AT_DetachPartition:
+			ATSimplePermissions(rel, ATT_TABLE);
+			/* No command-specific prep needed */
+			pass = AT_PASS_MISC;
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -3562,7 +3728,10 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
 			ATExecDropCluster(rel, lockmode);
 			break;
 		case AT_SetLogged:		/* SET LOGGED */
+			ATExecChangePersistence(rel, RELPERSISTENCE_PERMANENT);
+			break;
 		case AT_SetUnLogged:	/* SET UNLOGGED */
+			ATExecChangePersistence(rel, RELPERSISTENCE_UNLOGGED);
 			break;
 		case AT_AddOids:		/* SET WITH OIDS */
 			/* Use the ADD COLUMN code, unless prep decided to do nothing */
@@ -3670,6 +3839,12 @@ ATExecCmd(List **wqueue, AlteredTableInfo *tab, Relation rel,
 		case AT_GenericOptions:
 			ATExecGenericOptions(rel, (List *) cmd->def);
 			break;
+		case AT_AttachPartition:
+			ATExecAttachPartition(rel, (PartitionDef *) cmd->def);
+			break;
+		case AT_DetachPartition:
+			ATExecDetachPartition(rel, (PartitionDef *) cmd->def);
+			break;
 		default:				/* oops */
 			elog(ERROR, "unrecognized alter table type: %d",
 				 (int) cmd->subtype);
@@ -3701,8 +3876,12 @@ ATRewriteTables(AlterTableStmt *parsetree, List **wqueue, LOCKMODE lockmode)
 	{
 		AlteredTableInfo *tab = (AlteredTableInfo *) lfirst(ltab);
 
-		/* Foreign tables have no storage. */
-		if (tab->relkind == RELKIND_FOREIGN_TABLE)
+		/*
+		 * Foreign tables and partitioned tables have no storage; though in
+		 * the latter case, allow SET TABLESPACE.
+		 */
+		if (tab->relkind == RELKIND_FOREIGN_TABLE ||
+			(tab->relkind == RELKIND_PARTITIONED_REL && !tab->newTableSpace))
 			continue;
 
 		/*
@@ -4268,6 +4447,7 @@ ATSimplePermissions(Relation rel, int allowed_targets)
 	switch (rel->rd_rel->relkind)
 	{
 		case RELKIND_RELATION:
+		case RELKIND_PARTITIONED_REL:
 			actual_target = ATT_TABLE;
 			break;
 		case RELKIND_VIEW:
@@ -4439,6 +4619,40 @@ ATTypedTableRecursion(List **wqueue, Relation rel, AlterTableCmd *cmd,
 	}
 }
 
+/*
+ * ATPartitionedTableRecursion
+ *
+ * Propagate an ALTER TABLE command to all partitions of this relation
+ * including those of its partitions.
+ */
+static void
+ATPartitionedTableRecursion(List **wqueue, Relation rel, AlterTableCmd *cmd,
+					  LOCKMODE lockmode)
+{
+	/* Note that the recursion happens at the topmost parent only. */
+	if(rel->rd_rel->relkind == RELKIND_PARTITIONED_REL &&
+			!rel->rd_rel->relispartition)
+	{
+		ListCell   *cell;
+		List	   *partitions;
+
+		partitions = find_all_partitions(RelationGetRelid(rel), lockmode);
+
+		foreach(cell, partitions)
+		{
+			Oid			partid = lfirst_oid(cell);
+			Relation	partrel;
+
+			partrel = relation_open(partid, lockmode);
+
+			Assert(partrel->rd_rel->relispartition);
+
+			CheckTableNotInUse(partrel, "ALTER TABLE");
+			ATPrepCmd(wqueue, partrel, cmd, false, true, lockmode);
+			relation_close(partrel, NoLock);
+		}
+	}
+}
 
 /*
  * find_composite_type_dependencies
@@ -4498,6 +4712,7 @@ find_composite_type_dependencies(Oid typeOid, Relation origRelation,
 		att = rel->rd_att->attrs[pg_depend->objsubid - 1];
 
 		if (rel->rd_rel->relkind == RELKIND_RELATION ||
+			rel->rd_rel->relkind == RELKIND_PARTITIONED_REL ||
 			rel->rd_rel->relkind == RELKIND_MATVIEW)
 		{
 			if (origTypeName)
@@ -4660,6 +4875,20 @@ ATPrepAddColumn(List **wqueue, Relation rel, bool recurse, bool recursing,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("cannot add column to typed table")));
 
+	if (rel->rd_rel->relispartition && !recursing)
+	{
+		if(!strcmp(((ColumnDef *) cmd->def)->colname, "oid"))
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot add OID column to a partition of another table"),
+					 errhint("Perform SET WITH OIDS on the parent instead.")));
+		else
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot add column to a partition of another table"),
+					 errhint("Perform ADD COLUMN on the parent instead.")));
+	}
+
 	if (rel->rd_rel->relkind == RELKIND_COMPOSITE_TYPE)
 		ATTypedTableRecursion(wqueue, rel, cmd, lockmode);
 
@@ -5138,6 +5367,16 @@ ATPrepAddOids(List **wqueue, Relation rel, bool recurse, AlterTableCmd *cmd, LOC
 		cmd->subtype = AT_AddOidsRecurse;
 }
 
+static void
+ATPrepDropNotNull(Relation rel, bool recursing)
+{
+	if (rel->rd_rel->relispartition && !recursing)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot alter column of a partition of another table"),
+				 errhint("Perform ALTER COLUMN on the parent instead.")));
+}
+
 /*
  * ALTER TABLE ALTER COLUMN DROP NOT NULL
  *
@@ -5245,6 +5484,16 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
 	return address;
 }
 
+static void
+ATPrepSetNotNull(Relation rel, bool recursing)
+{
+	if (rel->rd_rel->relispartition && !recursing)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot alter column of a partition of another table"),
+				 errhint("Perform ALTER COLUMN on the parent instead.")));
+}
+
 /*
  * ALTER TABLE ALTER COLUMN SET NOT NULL
  *
@@ -5311,6 +5560,16 @@ ATExecSetNotNull(AlteredTableInfo *tab, Relation rel,
 	return address;
 }
 
+static void
+ATPrepColumnDefault(Relation rel, bool recursing)
+{
+	if (rel->rd_rel->relispartition && !recursing)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot alter column of a partition of another table"),
+				 errhint("Perform ALTER COLUMN on the parent instead.")));
+}
+
 /*
  * ALTER TABLE ALTER COLUMN SET/DROP DEFAULT
  *
@@ -5387,6 +5646,7 @@ ATPrepSetStatistics(Relation rel, const char *colName, Node *newValue, LOCKMODE
 	 * allowSystemTableMods to be turned on.
 	 */
 	if (rel->rd_rel->relkind != RELKIND_RELATION &&
+		rel->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 		rel->rd_rel->relkind != RELKIND_MATVIEW &&
 		rel->rd_rel->relkind != RELKIND_INDEX &&
 		rel->rd_rel->relkind != RELKIND_FOREIGN_TABLE)
@@ -5654,6 +5914,20 @@ ATPrepDropColumn(List **wqueue, Relation rel, bool recurse, bool recursing,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("cannot drop column from typed table")));
 
+	if (rel->rd_rel->relispartition && !recursing)
+	{
+		if(!strcmp(cmd->name, "oid"))
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot drop OID column of a partition of another table"),
+					 errhint("Perform SET WITHOUT OIDS on the parent instead.")));
+		else
+			ereport(ERROR,
+					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+					 errmsg("cannot drop column of a partition of another table"),
+					 errhint("Perform DROP COLUMN on the parent instead.")));
+	}
+
 	if (rel->rd_rel->relkind == RELKIND_COMPOSITE_TYPE)
 		ATTypedTableRecursion(wqueue, rel, cmd, lockmode);
 
@@ -5661,6 +5935,24 @@ ATPrepDropColumn(List **wqueue, Relation rel, bool recurse, bool recursing,
 		cmd->subtype = AT_DropColumnRecurse;
 }
 
+static bool
+find_att_reference_walker(Node *node, find_att_reference_context *context)
+{
+	if (node == NULL)
+		return false;
+
+	if (IsA(node, Var))
+	{
+		Var		   *variable = (Var *) node;
+		AttrNumber	attnum = variable->varattno;
+
+		if (attnum == context->attnum)
+			return true;
+	}
+
+	return expression_tree_walker(node, find_att_reference_walker, context);
+}
+
 /*
  * Return value is that of the dropped column.
  */
@@ -5719,6 +6011,41 @@ ATExecDropColumn(List **wqueue, Relation rel, const char *colName,
 				 errmsg("cannot drop inherited column \"%s\"",
 						colName)));
 
+	/* Don't drop if used in partition key */
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+	{
+		PartitionKeyInfo *key = BuildPartitionKeyInfo(rel);
+		int			i;
+		ListCell   *partexpr;
+
+		partexpr = list_head(key->pi_Expressions);
+		for (i = 0; i < key->pi_NumKeyAttrs; i++)
+		{
+			AttrNumber partatt = key->pi_KeyAttrNumbers[i];
+
+			if(partatt != 0)
+			{
+				if (attnum == partatt)
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+							 errmsg("cannot drop partition key column \"%s\"",
+							colName)));
+			}
+			else
+			{
+				find_att_reference_context context;
+
+				context.attnum = attnum;
+				if (find_att_reference_walker(lfirst(partexpr), &context))
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+							 errmsg("cannot drop column referenced in partition key expressions")));
+
+				partexpr = lnext(partexpr);
+			}
+		}
+	}
+
 	ReleaseSysCache(tuple);
 
 	/*
@@ -5990,6 +6317,24 @@ ATExecAddIndexConstraint(AlteredTableInfo *tab, Relation rel,
 	return address;
 }
 
+static void
+ATPrepAddConstraint(Relation rel)
+{
+	/*
+	 * It seems OK to disallow CHECK constraints on partitions. Any that
+	 * are necessary should be added to the parent. ExecConstraints()
+	 * should only ever look at the parent relation's constraints. Also,
+	 * validateConstraint() is taught to use parent's constraint tuples
+	 * during a ALTER TABLE VALIDATE CONSTRAINT on a partition or its
+	 * recursive application when performing the command on parent.
+	 */
+	if (rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot add constraints to a partition of another table"),
+				 errhint("Perform ADD CONSTRAINT on the parent instead.")));
+}
+
 /*
  * ALTER TABLE ADD CONSTRAINT
  *
@@ -6246,6 +6591,12 @@ ATAddForeignKeyConstraint(AlteredTableInfo *tab, Relation rel,
 	 * Validity checks (permission checks wait till we have the column
 	 * numbers)
 	 */
+	if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot reference relation \"%s\"", RelationGetRelationName(pkrel)),
+				 errdetail("Referencing partitioned tables in foreign key constraints is not supported.")));
+
 	if (pkrel->rd_rel->relkind != RELKIND_RELATION)
 		ereport(ERROR,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
@@ -6613,6 +6964,20 @@ ATAddForeignKeyConstraint(AlteredTableInfo *tab, Relation rel,
 	return address;
 }
 
+static void
+ATPrepAlterConstraint(Relation rel)
+{
+	/*
+	 * Remember there are no table constraints allowed on partitions
+	 * at all. There is nothing to alter in the first place.
+	 */
+	if (rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot alter constraints of a partition of another table"),
+				 errhint("Perform ALTER CONSTRAINT on the parent instead.")));
+}
+
 /*
  * ALTER TABLE ALTER CONSTRAINT
  *
@@ -6791,16 +7156,26 @@ ATExecValidateConstraint(Relation rel, char *constrName, bool recurse,
 	Form_pg_constraint con = NULL;
 	bool		found = false;
 	ObjectAddress address;
+	Oid			conrelid;
 
 	conrel = heap_open(ConstraintRelationId, RowExclusiveLock);
 
 	/*
+	 * A partition's table constraints are same as the parent's. So, scan
+	 * pg_constraint using parent OID instead of its own.
+	 */
+	if (rel->rd_rel->relispartition)
+		conrelid = get_partition_parent(rel->rd_id, true);
+	else
+		conrelid = RelationGetRelid(rel);
+
+	/*
 	 * Find and check the target constraint
 	 */
 	ScanKeyInit(&key,
 				Anum_pg_constraint_conrelid,
 				BTEqualStrategyNumber, F_OIDEQ,
-				ObjectIdGetDatum(RelationGetRelid(rel)));
+				ObjectIdGetDatum(conrelid));
 	scan = systable_beginscan(conrel, ConstraintRelidIndexId,
 							  true, NULL, 1, &key);
 
@@ -7302,6 +7677,14 @@ validateCheckConstraint(Relation rel, HeapTuple constrtup)
 	if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
 		return;
 
+	/*
+	 * There is nothing to do for a partitioned relation itself. If invoked
+	 * using ALTER TABLE VALIDATE CONSTRAINT, it has been arranged that the
+	 * command is applied to individual partitions where this could be done.
+	 */
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		return;
+
 	constrForm = (Form_pg_constraint) GETSTRUCT(constrtup);
 
 	estate = CreateExecutorState();
@@ -7374,6 +7757,14 @@ validateForeignKeyConstraint(char *conname,
 	Trigger		trig;
 	Snapshot	snapshot;
 
+	/*
+	 * There is nothing to do for a partitioned relation itself. If invoked
+	 * using ALTER TABLE VALIDATE CONSTRAINT, it has been arranged that the
+	 * command is applied to individual partitions where this could be done.
+	 */
+	if(rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		return;
+
 	ereport(DEBUG1,
 			(errmsg("validating foreign key constraint \"%s\"", conname)));
 
@@ -7621,6 +8012,20 @@ createForeignKeyTriggers(Relation rel, Oid refRelOid, Constraint *fkconstraint,
 						 indexOid, false);
 }
 
+static void
+ATPrepDropConstraint(Relation rel)
+{
+	/*
+	 * Remember there are no table constraints allowed on partitions
+	 * at all. There is nothing to drop in the first place.
+	 */
+	if (rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot drop constraints of a partition of another table"),
+				 errhint("Perform DROP CONSTRAINT on the parent instead.")));
+}
+
 /*
  * ALTER TABLE DROP CONSTRAINT
  *
@@ -7846,6 +8251,12 @@ ATPrepAlterColumnType(List **wqueue,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("cannot alter column type of typed table")));
 
+	if (rel->rd_rel->relispartition && !recursing)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot alter column type of a partition of another table"),
+				 errhint("Perform ALTER COLUMN TYPE on the parent instead.")));
+
 	/* lookup the attribute so we can check inheritance status */
 	tuple = SearchSysCacheAttName(RelationGetRelid(rel), colName);
 	if (!HeapTupleIsValid(tuple))
@@ -7870,6 +8281,41 @@ ATPrepAlterColumnType(List **wqueue,
 				 errmsg("cannot alter inherited column \"%s\"",
 						colName)));
 
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+	{
+		PartitionKeyInfo *key = BuildPartitionKeyInfo(rel);
+		int			i;
+		ListCell   *partexpr;
+
+		partexpr = list_head(key->pi_Expressions);
+		for (i = 0; i < key->pi_NumKeyAttrs; i++)
+		{
+			AttrNumber partatt = key->pi_KeyAttrNumbers[i];
+
+			if(partatt != 0)
+			{
+				if (attnum == partatt)
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+							 errmsg("cannot alter type of partition key column \"%s\"",
+							colName)));
+			}
+			else
+			{
+				find_att_reference_context context;
+
+				context.attnum = attnum;
+				if (find_att_reference_walker(lfirst(partexpr), &context))
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+							 errmsg("cannot alter type of a column referenced in partition key expressions")));
+
+				partexpr = lnext(partexpr);
+			}
+		}
+	}
+
+
 	/* Look up the target type */
 	typenameTypeIdAndMod(NULL, typeName, &targettype, &targettypmod);
 
@@ -7885,7 +8331,7 @@ ATPrepAlterColumnType(List **wqueue,
 					   list_make1_oid(rel->rd_rel->reltype),
 					   false);
 
-	if (tab->relkind == RELKIND_RELATION)
+	if (tab->relkind == RELKIND_RELATION || tab->relkind == RELKIND_PARTITIONED_REL)
 	{
 		/*
 		 * Set up an expression to transform the old data value to the new
@@ -8892,6 +9338,7 @@ ATExecChangeOwner(Oid relationOid, Oid newOwnerId, bool recursing, LOCKMODE lock
 	switch (tuple_class->relkind)
 	{
 		case RELKIND_RELATION:
+		case RELKIND_PARTITIONED_REL:
 		case RELKIND_VIEW:
 		case RELKIND_MATVIEW:
 		case RELKIND_FOREIGN_TABLE:
@@ -9355,6 +9802,7 @@ ATExecSetRelOptions(Relation rel, List *defList, AlterTableType operation,
 	switch (rel->rd_rel->relkind)
 	{
 		case RELKIND_RELATION:
+		case RELKIND_PARTITIONED_REL:
 		case RELKIND_TOASTVALUE:
 		case RELKIND_MATVIEW:
 			(void) heap_reloptions(rel->rd_rel->relkind, newOptions, true);
@@ -9514,9 +9962,6 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
 	Oid			newrelfilenode;
 	RelFileNode newrnode;
 	SMgrRelation dstrel;
-	Relation	pg_class;
-	HeapTuple	tuple;
-	Form_pg_class rd_rel;
 	ForkNumber	forkNum;
 	List	   *reltoastidxids = NIL;
 	ListCell   *lc;
@@ -9541,6 +9986,18 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
 	}
 
 	/*
+	 * For a partitioned table, setting pg_class.reltablespace suffices.
+	 */
+	if(rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+	{
+		/* XXX - keeping the relfilenode does not make any difference. */
+		SetRelationSpcFileNode(rel, newTableSpace, rel->rd_rel->relfilenode);
+		relation_close(rel, lockmode);
+
+		return;
+	}
+
+	/*
 	 * We cannot support moving mapped relations into different tablespaces.
 	 * (In particular this eliminates all shared catalogs.)
 	 */
@@ -9575,14 +10032,6 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
 		relation_close(toastRel, lockmode);
 	}
 
-	/* Get a modifiable copy of the relation's pg_class row */
-	pg_class = heap_open(RelationRelationId, RowExclusiveLock);
-
-	tuple = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(tableOid));
-	if (!HeapTupleIsValid(tuple))
-		elog(ERROR, "cache lookup failed for relation %u", tableOid);
-	rd_rel = (Form_pg_class) GETSTRUCT(tuple);
-
 	/*
 	 * Since we copy the file directly without looking at the shared buffers,
 	 * we'd better first flush out any pages of the source relation that are
@@ -9634,18 +10083,11 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
 	RelationDropStorage(rel);
 	smgrclose(dstrel);
 
-	/* update the pg_class row */
-	rd_rel->reltablespace = (newTableSpace == MyDatabaseTableSpace) ? InvalidOid : newTableSpace;
-	rd_rel->relfilenode = newrelfilenode;
-	simple_heap_update(pg_class, &tuple->t_self, tuple);
-	CatalogUpdateIndexes(pg_class, tuple);
+	/* Now update pg_class.reltablespace */
+	SetRelationSpcFileNode(rel, newTableSpace, newrelfilenode);
 
 	InvokeObjectPostAlterHook(RelationRelationId, RelationGetRelid(rel), 0);
 
-	heap_freetuple(tuple);
-
-	heap_close(pg_class, RowExclusiveLock);
-
 	relation_close(rel, NoLock);
 
 	/* Make sure the reltablespace change is visible */
@@ -9662,6 +10104,38 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
 }
 
 /*
+ * SetRelationTablespace - updates rel's reltablespace.
+ */
+static void
+SetRelationSpcFileNode(Relation rel, Oid newTableSpace, Oid newFileNode)
+{
+	Relation	pg_class;
+	HeapTuple	tuple;
+	Form_pg_class rd_rel;
+
+	/* Get a modifiable copy of the relation's pg_class row */
+	pg_class = heap_open(RelationRelationId, RowExclusiveLock);
+
+	tuple = SearchSysCacheCopy1(RELOID,
+							ObjectIdGetDatum(RelationGetRelid(rel)));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u",
+											RelationGetRelid(rel));
+	rd_rel = (Form_pg_class) GETSTRUCT(tuple);
+
+	/* update the pg_class row */
+	rd_rel->reltablespace = (newTableSpace == MyDatabaseTableSpace)
+										? InvalidOid : newTableSpace;
+	rd_rel->relfilenode = newFileNode;
+	simple_heap_update(pg_class, &tuple->t_self, tuple);
+	CatalogUpdateIndexes(pg_class, tuple);
+
+	heap_freetuple(tuple);
+	heap_close(pg_class, RowExclusiveLock);
+
+}
+
+/*
  * Alter Table ALL ... SET TABLESPACE
  *
  * Allows a user to move all objects of some type in a given tablespace in the
@@ -9768,7 +10242,8 @@ AlterTableMoveAll(AlterTableMoveAllStmt *stmt)
 
 		/* Only move the object type requested */
 		if ((stmt->objtype == OBJECT_TABLE &&
-			 relForm->relkind != RELKIND_RELATION) ||
+			 relForm->relkind != RELKIND_RELATION &&
+			 relForm->relkind != RELKIND_PARTITIONED_REL) ||
 			(stmt->objtype == OBJECT_INDEX &&
 			 relForm->relkind != RELKIND_INDEX) ||
 			(stmt->objtype == OBJECT_MATVIEW &&
@@ -9957,6 +10432,16 @@ ATPrepAddInherit(Relation child_rel)
 		ereport(ERROR,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("cannot change inheritance of typed table")));
+
+	if (child_rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot change inheritance of a partition of another table")));
+
+	if (child_rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot change inheritance of partitioned table")));
 }
 
 /*
@@ -10008,6 +10493,17 @@ ATExecAddInherit(Relation child_rel, RangeVar *parent, LOCKMODE lockmode)
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 		 errmsg("cannot inherit to temporary relation of another session")));
 
+	/* special case partitioned tables and partitions */
+	if (parent_rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				errmsg("cannot inherit from partitioned table \"%s\"", parent->relname)));
+
+	if (parent_rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				errmsg("cannot inherit from partition \"%s\"", parent->relname)));
+
 	/*
 	 * Check for duplicates in the list of parents, and determine the highest
 	 * inhseqno already present; we'll use the next one for the new parent.
@@ -10643,6 +11139,16 @@ ATExecAddOf(Relation rel, const TypeName *ofTypename, LOCKMODE lockmode)
 				typeobj;
 	HeapTuple	classtuple;
 
+	if (rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot attach a partition of another table to composite type")));
+
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot attach a partitioned table to composite type")));
+
 	/* Validate the type. */
 	typetuple = typenameType(NULL, ofTypename, NULL);
 	check_of_type(typetuple);
@@ -11067,6 +11573,189 @@ ATExecDisableRowSecurity(Relation rel)
 }
 
 /*
+ * ATExecAttachPartition
+ *
+ * CREATE TABLE ... PARTITION OF <parent> FOR VALUES ...
+ */
+static void
+ATExecAttachPartition(Relation parent, PartitionDef *partdef)
+{
+	Relation	child;
+	Datum	   *values;
+	int			listnvalues;
+	PartitionBoundInfo *partition = NULL;
+	ObjectAddress childobject,
+				parentobject;
+
+	child = heap_openrv(partdef->name, AccessExclusiveLock);
+
+	/* Cannot attach a partition to a non-partitioned table. */
+	if (parent->rd_rel->relkind != RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot create as partition of non-partitioned table \"%s\"",
+						RelationGetRelationName(parent))));
+	/*
+	 * Some persistence rules:
+	 *
+	 * 1. A permanent rel cannot be partition of a temporary one.
+	 */
+	if (parent->rd_rel->relpersistence == RELPERSISTENCE_TEMP &&
+			child->rd_rel->relpersistence != RELPERSISTENCE_TEMP)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+				 errmsg("cannot create as partition of temporary table \"%s\"",
+						RelationGetRelationName(parent))));
+
+	/* ... and if parent rel is temp, it must belong to this session. */
+	if (parent->rd_rel->relpersistence == RELPERSISTENCE_TEMP &&
+			!parent->rd_islocaltemp)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+				 errmsg("cannot create as partition of a temporary table of another session")));
+
+	/* 2. A permanent rel cannot be partition of an unlogged one. */
+	if (parent->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED &&
+			child->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+				 errmsg("cannot create as partition of unlogged table \"%s\"",
+						RelationGetRelationName(parent))));
+
+	/* 3. An unlogged rel cannot be partitin of a logged one. */
+	if (parent->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT &&
+			child->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+				 errmsg("cannot create as partition of logged table \"%s\"",
+						RelationGetRelationName(parent))));
+
+	/*
+	 * Now perform evaluation and initial validation of supplied
+	 * partition values
+	 */
+	partition = EvalPartitionBound(parent, partdef, &listnvalues);
+
+	/*
+	 * A new partition should not have overlapping definition with
+	 * an existing partition. If it does, error out.
+	 */
+	CheckPartitionAgainstExisting(parent, partition, &values);
+
+	StorePartitionBounds(child, parent, listnvalues, values);
+	SetRelationIsPartition(child, true);
+
+	/* Tell the world that child is now a partition of parent. */
+	CacheInvalidateRelcache(parent);
+
+	/*
+	 * Store a dependency too. This is to prevent parent from being
+	 * dropped without taking me too.
+	 */
+	parentobject.classId = RelationRelationId;
+	parentobject.objectId = RelationGetRelid(parent);
+	parentobject.objectSubId = 0;
+	childobject.classId = RelationRelationId;
+	childobject.objectId = RelationGetRelid(child);
+	childobject.objectSubId = 0;
+	recordDependencyOn(&childobject, &parentobject, DEPENDENCY_NORMAL);
+
+	heap_close(child, AccessExclusiveLock);
+}
+
+/*
+ * ATExecDetachPartition
+ *
+ * ALTER TABLE parent DETACH PARTITION partition;
+ */
+static void
+ATExecDetachPartition(Relation parent, PartitionDef *partition)
+{
+	Relation	child;
+	Oid			parentOid;
+	Oid			childOid;
+
+	/* cannot detach from a non-partitioned table */
+    if (parent->rd_rel->relkind != RELKIND_PARTITIONED_REL)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot detach from non-partitioned relation \"%s\"",
+						RelationGetRelationName(parent))));
+
+	child = heap_openrv(partition->name, AccessExclusiveLock);
+
+	if (!child->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot detach relation \"%s\"", RelationGetRelationName(child)),
+				 errdetail("\"%s\" is not a partition.", RelationGetRelationName(child))));
+
+	childOid = RelationGetRelid(child);
+	parentOid = RelationGetRelid(parent);
+
+	if (get_partition_parent(childOid, false) != parentOid)
+		ereport(ERROR,
+				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
+				 errmsg("cannot detach relation \"%s\"", RelationGetRelationName(child)),
+				 errdetail("\"%s\" is not a partition of \"%s\"",
+					RelationGetRelationName(child), RelationGetRelationName(parent))));
+
+	RemovePartitionBoundByRelId(childOid);
+	SetRelationIsPartition(child, false);
+	remove_partitioning_dependency(childOid, parentOid);
+
+	/* Tell the world that child is not a partition of parent anymore. */
+	CacheInvalidateRelcache(parent);
+
+
+	heap_close(child, AccessExclusiveLock);
+}
+
+/*
+ * remove_partition_dependency - removes dependency of parent on a partition
+ */
+static void
+remove_partitioning_dependency(Oid partitionOid, Oid parentOid)
+{
+	Relation	catalogRelation;
+	SysScanDesc scan;
+	ScanKeyData key[3];
+	HeapTuple	depTuple;
+
+	catalogRelation = heap_open(DependRelationId, RowExclusiveLock);
+
+	ScanKeyInit(&key[0],
+				Anum_pg_depend_classid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(RelationRelationId));
+	ScanKeyInit(&key[1],
+				Anum_pg_depend_objid,
+				BTEqualStrategyNumber, F_OIDEQ,
+				ObjectIdGetDatum(partitionOid));
+	ScanKeyInit(&key[2],
+				Anum_pg_depend_objsubid,
+				BTEqualStrategyNumber, F_INT4EQ,
+				Int32GetDatum(0));
+
+	scan = systable_beginscan(catalogRelation, DependDependerIndexId, true,
+							  NULL, 3, key);
+
+	while (HeapTupleIsValid(depTuple = systable_getnext(scan)))
+	{
+		Form_pg_depend dep = (Form_pg_depend) GETSTRUCT(depTuple);
+
+		if (dep->refclassid == RelationRelationId &&
+			dep->refobjid ==  parentOid &&
+			dep->refobjsubid == 0 &&
+			dep->deptype == DEPENDENCY_NORMAL)
+			simple_heap_delete(catalogRelation, &depTuple->t_self);
+	}
+
+	systable_endscan(scan);
+	heap_close(catalogRelation, RowExclusiveLock);
+}
+
+/*
  * ALTER FOREIGN TABLE <name> OPTIONS (...)
  */
 static void
@@ -11150,7 +11839,7 @@ ATExecGenericOptions(Relation rel, List *options)
  * checks are skipped), otherwise true.
  */
 static bool
-ATPrepChangePersistence(Relation rel, bool toLogged)
+ATPrepChangePersistence(Relation rel, bool toLogged, bool recursing)
 {
 	Relation	pg_constraint;
 	HeapTuple	tuple;
@@ -11158,6 +11847,15 @@ ATPrepChangePersistence(Relation rel, bool toLogged)
 	ScanKeyData skey[1];
 
 	/*
+	 * Cannot change persistence of partitions directly.
+	 */
+	if (rel->rd_rel->relispartition && !recursing)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+				 errmsg("cannot change persistence of a partition of another table"),
+				 errhint("Perform SET LOGGED/UNLOGGED on the parent instead.")));
+
+	/*
 	 * Disallow changing status for a temp table.  Also verify whether we can
 	 * get away with doing nothing; in such cases we don't need to run the
 	 * checks below, either.
@@ -11260,6 +11958,57 @@ ATPrepChangePersistence(Relation rel, bool toLogged)
 }
 
 /*
+ * ATExecChangePersistence - change pg_class.relpersistence of rel
+ *
+ * RELKIND_PARTITIONED_REL relations have no storage, so performing
+ * persistence change in ATRewriteTables step is not straightforward.
+ * Simply changing the relpersistence should suffice (?).
+ */
+static void
+ATExecChangePersistence(Relation rel, char persistence)
+{
+	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		SetRelationPersistence(RelationGetRelid(rel), persistence);
+}
+
+/*
+ * SetRelationPersistence - set relpersistence
+ */
+static void
+SetRelationPersistence(Oid relationId, char persistence)
+{
+	Relation	relationRelation;
+	HeapTuple	tuple;
+	Form_pg_class classtuple;
+
+	/*
+	 * Fetch a modifiable copy of the tuple, modify it, update pg_class.
+	 */
+	relationRelation = heap_open(RelationRelationId, RowExclusiveLock);
+	tuple = SearchSysCacheCopy1(RELOID, ObjectIdGetDatum(relationId));
+	if (!HeapTupleIsValid(tuple))
+		elog(ERROR, "cache lookup failed for relation %u", relationId);
+	classtuple = (Form_pg_class) GETSTRUCT(tuple);
+
+	if (classtuple->relpersistence != persistence)
+	{
+		classtuple->relpersistence = persistence;
+		simple_heap_update(relationRelation, &tuple->t_self, tuple);
+
+		/* keep the catalog indexes up to date */
+		CatalogUpdateIndexes(relationRelation, tuple);
+	}
+	else
+	{
+		/* no need to change tuple, but force relcache rebuild anyway */
+		CacheInvalidateRelcacheByTuple(tuple);
+	}
+
+	heap_freetuple(tuple);
+	heap_close(relationRelation, RowExclusiveLock);
+}
+
+/*
  * Execute ALTER TABLE SET SCHEMA
  */
 ObjectAddress
@@ -11352,6 +12101,7 @@ AlterTableNamespaceInternal(Relation rel, Oid oldNspOid, Oid nspOid,
 
 	/* Fix other dependent stuff */
 	if (rel->rd_rel->relkind == RELKIND_RELATION ||
+		rel->rd_rel->relkind == RELKIND_PARTITIONED_REL ||
 		rel->rd_rel->relkind == RELKIND_MATVIEW)
 	{
 		AlterIndexNamespaces(classRel, rel, oldNspOid, nspOid, objsMoved);
@@ -11794,8 +12544,8 @@ RangeVarCallbackOwnsTable(const RangeVar *relation,
 	relkind = get_rel_relkind(relId);
 	if (!relkind)
 		return;
-	if (relkind != RELKIND_RELATION && relkind != RELKIND_TOASTVALUE &&
-		relkind != RELKIND_MATVIEW)
+	if (relkind != RELKIND_RELATION && relkind != RELKIND_PARTITIONED_REL &&
+		relkind != RELKIND_TOASTVALUE && relkind != RELKIND_MATVIEW)
 		ereport(ERROR,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
 				 errmsg("\"%s\" is not a table or materialized view", relation->relname)));
@@ -11949,6 +12699,7 @@ RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid, Oid oldrelid,
 	 */
 	if (IsA(stmt, AlterObjectSchemaStmt) &&
 		relkind != RELKIND_RELATION &&
+		relkind != RELKIND_PARTITIONED_REL &&
 		relkind != RELKIND_VIEW &&
 		relkind != RELKIND_MATVIEW &&
 		relkind != RELKIND_SEQUENCE &&
@@ -11960,3 +12711,844 @@ RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid, Oid oldrelid,
 
 	ReleaseSysCache(tuple);
 }
+
+/*
+ * transformPartitionBy
+ *
+ * Transform any expressions present in the partition key.
+ */
+static PartitionBy *
+transformPartitionBy(Relation rel, PartitionBy *partitionby)
+{
+	ParseState 		*pstate = NULL;
+	RangeTblEntry	*rte;
+	ListCell		*l;
+	PartitionBy		*partby = NULL;
+
+	partby = (PartitionBy *) makeNode(PartitionBy);
+
+	partby->strategy = partitionby->strategy;
+	partby->location = partitionby->location;
+	partby->partParams = NIL;
+
+	/*
+	 * Create a dummy ParseState and insert the target relation as its sole
+	 * rangetable entry.  We need a ParseState for transformExpr.
+	 */
+	pstate = make_parsestate(NULL);
+	rte = addRangeTableEntryForRelation(pstate,
+										rel,
+										NULL,
+										false,
+										true);
+	addRTEtoQuery(pstate, rte, false, true, true);
+
+	/* take care of any partition expressions */
+	foreach(l, partitionby->partParams)
+	{
+		ListCell	   *column;
+		PartitionElem  *pelem = (PartitionElem *) lfirst(l);
+
+		/* Check for PARTITION BY ... ON (foo, foo) */
+		foreach(column, partby->partParams)
+		{
+			PartitionElem	*pparam = (PartitionElem *) lfirst(column);
+			if (pparam->name && strcmp(pelem->name, pparam->name) == 0)
+				ereport(ERROR,
+						(errcode(ERRCODE_DUPLICATE_COLUMN),
+						 errmsg("column \"%s\" appears twice in partition key", pelem->name),
+						 parser_errposition(pstate, pelem->location)));
+		}
+
+		if (pelem->expr)
+		{
+			/* Now do parse transformation of the expression */
+			pelem->expr = transformExpr(pstate, pelem->expr,
+										EXPR_KIND_PARTKEY_EXPRESSION);
+
+			/* we have to fix its collations too */
+			assign_expr_collations(pstate, pelem->expr);
+
+			/*
+			 * transformExpr() should have already rejected subqueries,
+			 * aggregates, and window functions, based on the EXPR_KIND_ for
+			 * a partition key expression.
+			 *
+			 * Also reject expressions returning sets; this is for consistency
+			 * DefineRelation() will make more checks.
+			 */
+			if (expression_returns_set(pelem->expr))
+				ereport(ERROR,
+						(errcode(ERRCODE_DATATYPE_MISMATCH),
+						 errmsg("partition key expression cannot return a set"),
+						 parser_errposition(pstate, pelem->location)));
+		}
+
+		partby->partParams = lappend(partby->partParams, pelem);
+	}
+
+	return partby;
+}
+
+/*
+ * ComputePartitionAttrs
+ *
+ * Compute per-partition-column information viz. partition column
+ * attribute numbers, expression, opclasses, etc.
+ */
+static PartitionKeyInfo *
+ComputePartitionAttrs(PartitionKeyInfo *pkinfo, Oid *partOpClassOids,
+						Oid relid, List *partParams)
+{
+	int			attn;
+	ListCell   *lc;
+
+	/* Initialize. */
+	pkinfo = makeNode(PartitionKeyInfo);
+
+	/* Process partColumns list */
+	attn = 0;
+	foreach(lc, partParams)
+	{
+		PartitionElem  *pelem = (PartitionElem *) lfirst(lc);
+		Oid				atttype;
+
+		if (pelem->name != NULL)
+		{
+			HeapTuple   atttuple;
+			Form_pg_attribute attform;
+
+			atttuple = SearchSysCacheAttName(relid, pelem->name);
+			if (!HeapTupleIsValid(atttuple))
+			{
+				/* difference in error message spellings is historical */
+				ereport(ERROR,
+						(errcode(ERRCODE_UNDEFINED_COLUMN),
+						 errmsg("column \"%s\" named in partition key does not exist",
+						 pelem->name)));
+			}
+			attform = (Form_pg_attribute) GETSTRUCT(atttuple);
+
+			if (attform->attnum <= 0)
+				ereport(ERROR,
+						(errcode(ERRCODE_UNDEFINED_COLUMN),
+						 errmsg("cannot use system column \"%s\" in partition key",
+						 pelem->name)));
+
+			pkinfo->pi_KeyAttrNumbers[attn] = attform->attnum;
+			atttype = attform->atttypid;
+			ReleaseSysCache(atttuple);
+		}
+		else
+		{
+			/* Partition key expression */
+			Node	   *expr = pelem->expr;
+
+			Assert(expr != NULL);
+			atttype = exprType(expr);
+
+			if (IsA(expr, CollateExpr))
+				ereport(ERROR,
+						(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+						 errmsg("cannot use COLLATE in partition key expression")));
+
+			if (IsA(expr, Var) &&
+				((Var *) expr)->varattno != InvalidAttrNumber)
+			{
+				/*
+				 * User wrote "(column)" or "(column COLLATE something)".
+				 * Treat it like simple attribute anyway.
+				 */
+				pkinfo->pi_KeyAttrNumbers[attn] = ((Var *) expr)->varattno;
+			}
+			else
+			{
+				pkinfo->pi_KeyAttrNumbers[attn] = 0; /* marks expression */
+				pkinfo->pi_Expressions = lappend(pkinfo->pi_Expressions, expr);
+
+				/*
+				 * transformExpr() should have already rejected subqueries,
+				 * aggregates, and window functions, based on the EXPR_KIND_
+				 * for a partition key expression.
+				 */
+
+				/*
+				 * An expression using mutable functions is probably wrong even
+				 * even to use in a partition key
+				 */
+				expr = (Node *) expression_planner((Expr *) expr);
+
+				if (IsA(expr, Const))
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("cannot use a constant expression as partition key")));
+
+				if (contain_mutable_functions(expr))
+					ereport(ERROR,
+							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+							 errmsg("functions in partition key expression must be marked IMMUTABLE")));
+			}
+		}
+
+		/*
+		 * Identify the opclass to use. At the moment, we use "btree" operators
+		 * that seems enough for list and range partitioning.
+		 * XXX - To be truly extensible, we should provide means of specifying
+		 * which AM to use (along with the opclass).
+		 */
+		partOpClassOids[attn++] = GetIndexOpClass(pelem->opclass,
+										  atttype,
+										  "btree",
+										  BTREE_AM_OID);
+	}
+
+	pkinfo->pi_NumKeyAttrs = attn;
+	return pkinfo;
+}
+
+/*
+ * EvalPartitionBound - validate and evalulate partition bounds specified
+ * in FOR VALUES clause of a partition definition.
+ */
+static PartitionBoundInfo *
+EvalPartitionBound(Relation parent, PartitionDef *partdef, int *listnvalues)
+{
+	int					rangenmins, rangenmaxs;
+	char				partstrategy;
+	int					partnatts;
+	FmgrInfo		   *partattcmpfn;
+	int32				cmpval;
+	PartitionValues	   *values;
+	PartitionBoundInfo *result;
+	PartitionKeyTypeInfo *typinfo = get_key_type_info(parent);
+
+	partstrategy = parent->rd_pkey->partstrategy;
+	partnatts = parent->rd_pkey->partnatts;
+	partattcmpfn = parent->rd_partsupfuncs;
+
+	result = (PartitionBoundInfo *) palloc0(sizeof(PartitionBoundInfo));
+	result->oid = InvalidOid;	/* as yet, let's assume */
+	result->partnatts = partnatts;
+
+	/* Transform and evaluate value expressions for this partition */
+	values = partdef->values;
+	switch (partstrategy)
+	{
+		case PARTITION_STRAT_LIST:
+			/* CREATE TABLE ... PARTITION BY LIST ON ... enforces */
+			Assert(partnatts == 1);
+
+			if (!values->listvalues)
+				ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("invalid FOR VALUES specification for a list partition")));
+
+			result->listnvalues = *listnvalues = list_length(values->listvalues);
+
+			Assert (result->listnvalues >= 1);	/* Grammar enforces. */
+
+			/*
+			 * Currently only one column is supported here.
+			 *
+			 * datum[0] - list of allowed values for the only key column
+			 */
+
+			result->listvalues = evalPartitionListBound(values->listvalues,
+															typinfo, result->listnvalues);
+
+			/* TODO: uniqueify(result->listvalues) */
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			if (!values->rangemins && !values->rangemaxs)
+				ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("invalid FOR VALUES specification for a range partition")));
+
+			rangenmins = list_length(values->rangemins);
+			rangenmaxs = list_length(values->rangemaxs);
+
+			if ((values->rangemins && rangenmins != partnatts) ||
+				(values->rangemaxs && rangenmaxs != partnatts))
+				ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("range partition FOR VALUES specification does not match the partition key")));
+
+			result->rangeminnull = (values->rangemins == NIL);
+			result->rangemaxnull = (values->rangemaxs == NIL);
+
+			result->rangemins = evalPartitionRangeBound(values->rangemins, typinfo, partnatts);
+			result->rangemaxs = evalPartitionRangeBound(values->rangemaxs, typinfo, partnatts);
+
+			/*
+			 * Safeguard against empty partition definitions
+			 */
+			if (result->rangemins && result->rangemaxs)
+			{
+				cmpval = range_partition_cmp_bounds(result->rangemins,
+													result->rangemaxs,
+													partnatts, partattcmpfn);
+				if (cmpval >= 0)
+					ereport(ERROR,
+						(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+						 errmsg("cannot define partition with an empty range")));
+			}
+			break;
+	}
+
+	pfree(typinfo);
+	return result;
+}
+
+/*
+ * evalPartitionListBound - evaluate a list of expressions to build a datum
+ * array corresponding to a list partition bound.
+ */
+static Datum *
+evalPartitionListBound(List *values, PartitionKeyTypeInfo *typinfo, int nvalues)
+{
+	ListCell	*cell;
+	ParseState	*pstate;
+	EState		*estate;
+	ExprContext	*ecxt;
+	int			i;
+	Datum		*datum;
+
+	if (!values)
+		return NULL;
+
+	datum = (Datum *) palloc(nvalues * sizeof(Datum));
+	pstate = make_parsestate(NULL);
+	estate = CreateExecutorState();
+	ecxt = GetPerTupleExprContext(estate);
+
+	i = 0;
+	foreach(cell, values)
+	{
+		Node		*value = (Node *) lfirst(cell);
+		bool		isnull;
+		ExprState	*expr;
+		MemoryContext	oldcxt;
+		Oid			valuetype;
+
+		oldcxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
+
+		valuetype = exprType(value);
+		value = coerce_to_target_type(NULL,
+									value, valuetype,
+									typinfo->typid[0],
+									typinfo->typmod[0],
+									COERCION_ASSIGNMENT,
+									COERCE_IMPLICIT_CAST,
+									-1);
+		if (value == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_DATATYPE_MISMATCH),
+					 errmsg("specified values in FOR VALUES do not match the partition key"),
+					 errhint("You will need to rewrite or cast the expression.")));
+
+		expr = ExecPrepareExpr((Expr *) value, estate);
+
+		datum[i] = ExecEvalExpr(expr, ecxt, &isnull, NULL);
+		if (isnull)
+			ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					errmsg("a partition bound value must not be NULL")));
+
+		MemoryContextSwitchTo(oldcxt);
+
+		if (!typinfo->typbyval[0])
+		{
+			if (typinfo->typlen[0] == -1)
+				datum[i] = PointerGetDatum(PG_DETOAST_DATUM_COPY(datum[i]));
+			else
+				datum[i] = datumCopy(datum[i], false, typinfo->typlen[0]);
+		}
+
+		ResetPerTupleExprContext(estate);
+		i++;
+	}
+
+	return datum;
+}
+
+/*
+ * evalPartitionRangeBound - evaluate a list of expressions to build a datum
+ * array corresponding to a range partition bound.
+ */
+static Datum *
+evalPartitionRangeBound(List *values, PartitionKeyTypeInfo *typinfo, int partnatts)
+{
+	ListCell	*cell;
+	ParseState	*pstate;
+	EState		*estate;
+	ExprContext	*ecxt;
+	int			i;
+	Datum		*datum;
+
+	if (!values)
+		return NULL;
+
+	datum = (Datum *) palloc(partnatts * sizeof(Datum));
+	pstate = make_parsestate(NULL);
+	estate = CreateExecutorState();
+	ecxt = GetPerTupleExprContext(estate);
+
+	i = 0;
+	foreach(cell, values)
+	{
+		Node		*value = (Node *) lfirst(cell);
+		bool		isnull;
+		ExprState	*expr;
+		MemoryContext	oldcxt;
+		Oid			valuetype;
+
+		oldcxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
+
+		valuetype = exprType(value);
+
+		value = coerce_to_target_type(NULL,
+									value, valuetype,
+									typinfo->typid[i],
+									typinfo->typmod[i],
+									COERCION_ASSIGNMENT,
+									COERCE_IMPLICIT_CAST,
+									-1);
+		if (value == NULL)
+			ereport(ERROR,
+					(errcode(ERRCODE_DATATYPE_MISMATCH),
+					 errmsg("specified values in FOR VALUES do not match the partition key"),
+					 errhint("You will need to rewrite or cast the expression.")));
+
+		expr = ExecPrepareExpr((Expr *) value, estate);
+
+		datum[i] = ExecEvalExpr(expr, ecxt, &isnull, NULL);
+		if (isnull)
+			ereport(ERROR,
+				(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					errmsg("a partition value must not be NULL")));
+
+		MemoryContextSwitchTo(oldcxt);
+
+		if (!typinfo->typbyval[i])
+		{
+			if (typinfo->typlen[i] == -1)
+				datum[i] = PointerGetDatum(PG_DETOAST_DATUM_COPY(datum[i]));
+			else
+				datum[i] = datumCopy(datum[i], false, typinfo->typlen[i]);
+		}
+
+		ResetPerTupleExprContext(estate);
+		i++;
+	}
+
+	return datum;
+}
+
+/* typinfo->typbyval[], typinfo->typlen[] better be in place */
+#define COPY_RANGE_PARTITION_BOUND(dest, src, natts) \
+do { \
+	int	j; \
+	(dest) = (Datum *) palloc0((natts) * sizeof(Datum)); \
+	for (j = 0; j < (natts); j++) \
+	{ \
+		(dest)[j] = typinfo->typbyval[j] \
+					? (src)[j] \
+					: ((typinfo->typlen[j] == -1) \
+						? PointerGetDatum(PG_DETOAST_DATUM_COPY((src)[j])) \
+						: datumCopy((src)[j], false, typinfo->typlen[j])); \
+	} \
+} while(0);
+
+/*
+ * CheckPartitionAgainstExisting
+ *
+ * Does the new partition overlap with an existing partition of parent?
+ *
+ * Also if either new->rangemins or new->rangemaxs are omitted, we
+ * substitute the missing bound by locating a proper adjacent partition
+ * and use one of its bounds.
+ */
+static void
+CheckPartitionAgainstExisting(Relation parent,
+								PartitionBoundInfo *new,
+								Datum **final)
+{
+	int		numparts;
+
+	char			strategy = parent->rd_pkey->partstrategy;
+	int				partnatts = parent->rd_pkey->partnatts;
+	FmgrInfo	   *partattcmpfn = parent->rd_partsupfuncs;
+	PartitionBoundInfo **existing = GetPartitionBounds(parent, &numparts);
+	PartitionBoundInfo **newlist = NULL;
+
+	/* Type info for partition key attributes */
+	PartitionKeyTypeInfo *typinfo = get_key_type_info(parent);
+
+	Datum		   *newbound;
+	int				i, newpos;
+	int32			cmpval;
+	bool			overlaps = false;
+	Oid				overlapsWith = InvalidOid;
+
+	/* Trivial when new is the first partition of parent. */
+	if (!existing)
+		goto cleanup;
+
+	/* Pretty trivial. */
+	if (strategy == PARTITION_STRAT_LIST)
+	{
+		Assert(new->listvalues);
+
+		for (i = 0; i < numparts; i++)
+		{
+			if (list_partition_overlaps(new, existing[i], partattcmpfn))
+			{
+				overlaps = true;
+				overlapsWith = existing[i]->oid;
+				goto cleanup;
+			}
+		}
+
+		/* No existing partition had an overlapping list of values. */
+		goto cleanup;
+	}
+
+	/* Range partitioning case is bit more complicated. */
+	newlist = (PartitionBoundInfo **) repalloc(existing, (numparts + 1)
+												* sizeof(PartitionBoundInfo *));
+	newlist[numparts] = new;
+
+	/*
+	 * Sorting the newlist would put the new partition into its would-be
+	 * position. Code below then checks if being in that positions causes
+	 * it to overlap with an adjacent partition. If it does, complain!
+	 *
+	 * Existing partitions are known ordered such that:
+	 *
+	 * (i-1).min < (i-1).max <= (i).min < (i).max <= (i+1).min < (i+1).max
+	 *
+	 * Position assigned here depends on the ascending order of either
+	 * max or min. Code below makes sure that the remaining bound doesn't
+	 * cross a bound to cause an overlap. In other words, based on whether
+	 * min or max bound was used for ordering partitions above, the following
+	 * is implied: new.min <= right.min and new.max >= left.max, respectively.
+	 *
+	 * Note: qsort_arg()'s range partition ordering callback functions are
+	 * written such that if new.max == some_existing.max, new is ordered
+	 * before the existing. Reverse for the new.min == some_existing.min case.
+	 * It uses the fact that new.oid = InvalidOid to make that call. That is
+	 * is simply to make qsort's ordering result look less than arbitrary for
+	 * such cases. That behavior determines the result of some overlap tests
+	 * below.
+	 */
+	if (!new->rangemaxnull)
+		qsort_arg(newlist, numparts+1, sizeof(PartitionBoundInfo *),
+											range_partition_cmp_max,
+											partattcmpfn);
+	else
+		qsort_arg(newlist, numparts+1, sizeof(PartitionBoundInfo *),
+											range_partition_cmp_min,
+											partattcmpfn);
+	for (i = 0; i < numparts+1; i++)
+	{
+		if (newlist[i]->oid == InvalidOid)
+		{
+			newpos = i;
+			break;
+		}
+	}
+
+	/*
+	 * Compare new with adjacents for overlap; new overlaps if:
+	 *
+	 * On left: new.max == left.max || new.min < left.max
+	 * On right: new.min == right.min  || new.max > right.min
+	 */
+
+	/*
+	 * newbound is a shorthand for which of new's bounds is used in overlap
+	 * tests below. It is new->rangemaxs unless the new partition is defined
+	 * to have null rangemax.
+	 */
+	newbound = !new->rangemaxnull ? new->rangemaxs : new->rangemins;
+
+	/*
+	 * Cases (new.maxnull AND left.maxnull) and (new.minnull AND right.minnull)
+	 * can be dealt with quickly. We allow only one leftmost partition with
+	 * NULL min and only one rightmost partition with NULL max.
+	 *
+	 * Case 1: It's possible that the new partition of definition [min, NULL)
+	 * overlaps with the rightmost partition with NULL max bound.
+	 */
+	if (new->rangemaxnull
+			&& (newpos > 0) && newlist[newpos-1]->rangemaxnull)
+	{
+		overlaps = true;
+		overlapsWith = newlist[newpos-1]->oid;
+		goto cleanup;
+	}
+
+	/*
+	 * Case 2: It's possible that the new partition of definition (NULL, max]
+	 * overlaps with the leftmost partition with NULL min bound.
+	 */
+	if (new->rangeminnull
+			&& (newpos <= (numparts-1)) && newlist[newpos+1]->rangeminnull)
+	{
+		overlaps = true;
+		overlapsWith = newlist[newpos+1]->oid;
+		goto cleanup;
+	}
+
+	/*
+	 * Overlap the left adjacent partition?
+	 * If new.minnull, we might be able to find a value for it here.
+	 */
+	if (newpos > 0 )
+	{
+		cmpval = range_partition_cmp_bounds(newbound,
+										newlist[newpos-1]->rangemaxs,
+										partnatts, partattcmpfn);
+		/* new.min/max == left.max */
+		if (!cmpval && !new->rangemaxnull)
+		{
+			overlaps = true;
+			overlapsWith = newlist[newpos-1]->oid;
+			goto cleanup;
+		}
+
+		/* new.min < left.max */
+		if (cmpval < 0)
+		{
+			overlaps = true;
+			overlapsWith = newlist[newpos-1]->oid;
+			goto cleanup;
+		}
+
+		/* new.min/max > left.max */
+		if (new->rangeminnull)
+		{
+			/*
+			 * If there is room between newpos-1 and newpos+1, acquire
+			 * (newpos-1)'s rangemaxs as rangemins for the new partition.
+			 */
+			cmpval = (newpos < numparts) ?
+					range_partition_cmp_bounds(newlist[newpos-1]->rangemaxs,
+											newlist[newpos+1]->rangemins,
+											partnatts, partattcmpfn)
+						/* new is placed last. */
+						: range_partition_cmp_bounds(newbound,
+											newlist[newpos-1]->rangemaxs,
+											partnatts, partattcmpfn);
+			if (!cmpval)
+			{
+				/* There is no room at all to fill. */
+				overlaps = true;
+				overlapsWith = newlist[newpos+1]->oid;
+				goto cleanup;
+			}
+			else
+			{
+				/*
+				 * new.max > left.max OR left.max < right.min.
+				 *
+				 * So, we can use left.max as new.min unless new.max happens
+				 * to overlap with right which we check later.
+				 */
+				COPY_RANGE_PARTITION_BOUND(new->rangemins,
+											newlist[newpos-1]->rangemaxs,
+											partnatts);
+				new->rangeminnull = false;
+			}
+		}
+		else
+		{
+			/*
+			 * Better make sure new->min doesn't enter (newpos-1)'s range.
+			 */
+			cmpval = range_partition_cmp_bounds(newlist[newpos]->rangemins,
+											newlist[newpos-1]->rangemaxs,
+											partnatts, partattcmpfn);
+
+			if (cmpval < 0)
+			{
+				overlaps = true;
+				overlapsWith = newlist[newpos-1]->oid;
+				goto cleanup;
+			}
+		}
+	}
+
+	/*
+	 * Overlap the right adjacent partition?
+	 * If new.maxnull, we might be able to find a value for it here.
+	 */
+	if (newpos < numparts)
+	{
+		cmpval = range_partition_cmp_bounds(newbound,
+										newlist[newpos+1]->rangemins,
+										partnatts, partattcmpfn);
+
+		/* new.min/max == right.min */
+		if (!cmpval && new->rangemaxnull)
+		{
+			overlaps = true;
+			overlapsWith = newlist[newpos+1]->oid;
+			goto cleanup;
+		}
+
+		/* new.max > right.min */
+		if (cmpval > 0)
+		{
+			overlaps = true;
+			overlapsWith = newlist[newpos+1]->oid;
+			goto cleanup;
+		}
+
+		/* new.min/max < right.min */
+		if (new->rangemaxnull)
+		{
+			/*
+			 * If there is room between newpos-1 and newpos+1, acquire
+			 * (newpos+1)'s rangemins as rangemaxs for the new partition.
+			 */
+			cmpval = (newpos > 0) ?
+					range_partition_cmp_bounds(newlist[newpos-1]->rangemaxs,
+											newlist[newpos+1]->rangemins,
+											partnatts, partattcmpfn)
+						/* new is placed first. */
+						: range_partition_cmp_bounds(newbound,
+											newlist[newpos+1]->rangemins,
+											partnatts, partattcmpfn);
+			if (!cmpval)
+			{
+				/* There is no room at all to fill. */
+				overlaps = true;
+				overlapsWith = newlist[newpos-1]->oid;
+				goto cleanup;
+			}
+			else
+			{
+				/*
+				 * new.min < right.min OR left.max < right.min.
+				 *
+				 * So, we can use right.max as new.max unless new.min happens
+				 * to overlap with left which is something we ruled out above.
+				 */
+				COPY_RANGE_PARTITION_BOUND(new->rangemaxs,
+											newlist[newpos+1]->rangemins,
+											partnatts);
+				new->rangemaxnull = false;
+			}
+		}
+		else
+		{
+			/*
+			 * Better make sure new->max doesn't enter (newpos+1)'s range.
+			 */
+			cmpval = range_partition_cmp_bounds(newlist[newpos]->rangemaxs,
+											newlist[newpos+1]->rangemins,
+											 partnatts, partattcmpfn);
+			if (cmpval > 0)
+			{
+				overlaps = true;
+				overlapsWith = newlist[newpos+1]->oid;
+				goto cleanup;
+			}
+		}
+	}
+
+cleanup:
+	if (existing)
+	{
+		if (newlist)
+			free_partitions(newlist, numparts+1);
+		else
+			free_partitions(existing, numparts);
+	}
+
+	if (overlaps)
+		ereport(ERROR,
+			(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+			 errmsg("cannot create partition that overlaps with an existing one"),
+			 errdetail("New partition's definition overlaps with existing partition \"%s\" of \"%s\".",
+						get_rel_name(overlapsWith), RelationGetRelationName(parent))));
+
+	/*
+	 * Now construct the datum array that will be stored into pg_partition.
+	 */
+	switch (strategy)
+	{
+		case PARTITION_STRAT_LIST:
+			*final = (Datum *) palloc0(new->listnvalues * sizeof(Datum));
+
+			for (i = 0; i < new->listnvalues; i++)
+				(*final)[i] = new->listvalues[i];
+
+			pfree(new->listvalues);
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			*final = (Datum *) palloc0(partnatts * sizeof(Datum));
+			for (i = 0; i < partnatts; i++)
+			{
+				Datum 					datums[2];
+				bool					nulls[2];
+				int						dims[1];
+				int						lbs[1];
+				ArrayType  			   *rangebounds;
+
+				if (new->rangeminnull)
+					nulls[0] = true;
+				else
+				{
+					datums[0] = new->rangemins[i];
+					nulls[0] = false;
+				}
+
+				if (new->rangemaxnull)
+					nulls[1] = true;
+				else
+				{
+					datums[1] = new->rangemaxs[i];
+					nulls[1] = false;
+				}
+
+				dims[0] = 2;
+				lbs[0] = 1;
+
+				rangebounds = construct_md_array(datums, nulls, 1, dims, lbs,
+												typinfo->typid[i], typinfo->typlen[i],
+												typinfo->typbyval[i], typinfo->typalign[i]);
+
+				(*final)[i] = PointerGetDatum(rangebounds);
+			}
+
+			if (new->rangemins)
+				pfree(new->rangemins);
+			if (new->rangemaxs)
+				pfree(new->rangemaxs);
+
+			break;
+	}
+
+	pfree(new);
+	pfree(typinfo);
+}
+
+/*
+ * RangeVarGetTablespace - returns relation's tablespace OID.
+ */
+static Oid
+RangeVarGetTablespace(RangeVar *relation)
+{
+	Relation	rel;
+	Oid			tablespaceId;
+
+	rel = heap_openrv(relation, NoLock);
+	tablespaceId = rel->rd_rel->reltablespace;
+	heap_close(rel, NoLock);
+
+	return tablespaceId;
+}
diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c
index d169027..cac8934 100644
--- a/src/backend/commands/trigger.c
+++ b/src/backend/commands/trigger.c
@@ -169,11 +169,17 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 	else
 		rel = heap_openrv(stmt->relation, ShareRowExclusiveLock);
 
+	if (rel->rd_rel->relispartition)
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("cannot define triggers on a partition of another table"),
+				 errhint("Perform CREATE TRIGGER on the parent instead.")));
 	/*
 	 * Triggers must be on tables or views, and there are additional
 	 * relation-type-specific restrictions.
 	 */
-	if (rel->rd_rel->relkind == RELKIND_RELATION)
+	if (rel->rd_rel->relkind == RELKIND_RELATION ||
+		rel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
 	{
 		/* Tables can't have INSTEAD OF triggers */
 		if (stmt->timing != TRIGGER_TYPE_BEFORE &&
@@ -183,6 +189,13 @@ CreateTrigger(CreateTrigStmt *stmt, const char *queryString,
 					 errmsg("\"%s\" is a table",
 							RelationGetRelationName(rel)),
 					 errdetail("Tables cannot have INSTEAD OF triggers.")));
+
+		/* Disallow row-level AFTER triggers on partitioned tables, *yet*. */
+		if (rel->rd_rel->relkind == RELKIND_PARTITIONED_REL &&
+				stmt->timing == TRIGGER_TYPE_AFTER && stmt->row)
+			ereport(ERROR,
+					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+					 errmsg("Row-level AFTER triggers are not supported on partitioned tables")));
 	}
 	else if (rel->rd_rel->relkind == RELKIND_VIEW)
 	{
@@ -1113,6 +1126,7 @@ RemoveTriggerById(Oid trigOid)
 	rel = heap_open(relid, AccessExclusiveLock);
 
 	if (rel->rd_rel->relkind != RELKIND_RELATION &&
+		rel->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 		rel->rd_rel->relkind != RELKIND_VIEW &&
 		rel->rd_rel->relkind != RELKIND_FOREIGN_TABLE)
 		ereport(ERROR,
@@ -1218,7 +1232,9 @@ RangeVarCallbackForRenameTrigger(const RangeVar *rv, Oid relid, Oid oldrelid,
 	form = (Form_pg_class) GETSTRUCT(tuple);
 
 	/* only tables and views can have triggers */
-	if (form->relkind != RELKIND_RELATION && form->relkind != RELKIND_VIEW &&
+	if (form->relkind != RELKIND_RELATION &&
+		form->relkind != RELKIND_PARTITIONED_REL &&
+		form->relkind != RELKIND_VIEW &&
 		form->relkind != RELKIND_FOREIGN_TABLE)
 		ereport(ERROR,
 				(errcode(ERRCODE_WRONG_OBJECT_TYPE),
diff --git a/src/backend/commands/typecmds.c b/src/backend/commands/typecmds.c
index de91353..6478a97 100644
--- a/src/backend/commands/typecmds.c
+++ b/src/backend/commands/typecmds.c
@@ -2900,6 +2900,7 @@ get_rels_with_domain(Oid domainOid, LOCKMODE lockmode)
 			 * index columns.
 			 */
 			if (rel->rd_rel->relkind != RELKIND_RELATION &&
+				rel->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 				rel->rd_rel->relkind != RELKIND_MATVIEW)
 			{
 				relation_close(rel, lockmode);
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 85b0483..6f71ffb 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -1292,6 +1292,21 @@ vacuum_rel(Oid relid, RangeVar *relation, int options, VacuumParams *params)
 	}
 
 	/*
+	 * XXX - Vacuuming a partitioned table not supported yet
+	 */
+	if (onerel->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+	{
+		if(!IsAutoVacuumWorkerProcess())
+			ereport(ERROR,
+					(errmsg("cannot vacuum partitioned table \"%s\"", RelationGetRelationName(onerel)),
+					 errhint("Run VACUUM on individual partitions one-by-one.")));
+		else
+			ereport(WARNING,
+					(errmsg("skipping \"%s\" --- cannot vacuum partitioned tables",
+							RelationGetRelationName(onerel))));
+	}
+
+	/*
 	 * Check that it's a vacuumable relation; we used to do this in
 	 * get_rel_oids() but seems safer to check after we've locked the
 	 * relation.
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2c65a90..3973c4d 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -56,6 +56,7 @@
 #include "utils/acl.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partition.h"
 #include "utils/rls.h"
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
@@ -1017,6 +1018,36 @@ CheckValidResultRel(Relation resultRel, CmdType operation)
 	switch (resultRel->rd_rel->relkind)
 	{
 		case RELKIND_RELATION:
+			/*
+			 * prevent direct modifications on partitions; we have no way
+			 * of ensuring at partition level that paritioning rules allow
+			 * the result row to be in resultRel at all
+			 */
+			if (resultRel->rd_rel->relispartition)
+			{
+				switch (operation)
+				{
+					case CMD_INSERT:
+						ereport(ERROR,
+							(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+							 errmsg("cannot insert into a partition of another table",
+							 RelationGetRelationName(resultRel)),
+							 errhint("Perform INSERT on the parent instead.")));
+						break;
+					case CMD_UPDATE:
+						ereport(ERROR,
+							(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+							 errmsg("cannot update a partition of another table",
+							 RelationGetRelationName(resultRel)),
+							 errhint("Perform UPDATE on the parent instead.")));
+						break;
+					default:
+						/* CMD_DELETE doesn't hurt */
+						break;
+				}
+			}
+			break;
+		case RELKIND_PARTITIONED_REL:
 			/* OK */
 			break;
 		case RELKIND_SEQUENCE:
@@ -1152,6 +1183,9 @@ CheckValidRowMarkRel(Relation rel, RowMarkType markType)
 		case RELKIND_RELATION:
 			/* OK */
 			break;
+		case RELKIND_PARTITIONED_REL:
+			/* OK? should never be? */
+			break;
 		case RELKIND_SEQUENCE:
 			/* Must disallow this because we don't vacuum sequences */
 			ereport(ERROR,
@@ -1246,6 +1280,11 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo,
 	resultRelInfo->ri_ConstraintExprs = NULL;
 	resultRelInfo->ri_junkFilter = NULL;
 	resultRelInfo->ri_projectReturning = NULL;
+	if (resultRelationDesc->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+	{
+		resultRelInfo->ri_PartitionKeyInfo = BuildPartitionKeyInfo(resultRelationDesc);
+		resultRelInfo->ri_Partitions = RelationGetPartitionInfo(resultRelationDesc);
+	}
 }
 
 /*
@@ -2880,3 +2919,62 @@ EvalPlanQualEnd(EPQState *epqstate)
 	epqstate->planstate = NULL;
 	epqstate->origslot = NULL;
 }
+
+/*
+ * ExecFindPartition - determine the partition for tuple contained in slot
+ *
+ * Based on resultRelInfo->ri_Partitions, find_partition_for_tuple() determines
+ * oid of the target partition. If it turns out to be InvalidOid, complain that
+ * the tuple cannot be inserted in this relation.
+ */
+ResultRelInfo *
+ExecFindPartition(ResultRelInfo *resultRelInfo, TupleTableSlot *slot,
+					EState *estate)
+{
+	Oid				partOid;
+	Relation		parentRelDesc = resultRelInfo->ri_RelationDesc;
+	Relation		partRelDesc;
+	ResultRelInfo  *partition;
+
+	partOid = find_partition_for_tuple(parentRelDesc,
+										resultRelInfo->ri_PartitionKeyInfo,
+										resultRelInfo->ri_Partitions,
+										slot,
+										estate);
+
+	if (partOid != InvalidOid)
+	{
+		partRelDesc = heap_open(partOid, RowExclusiveLock);
+		partition = makeNode(ResultRelInfo);
+		InitResultRelInfo(partition,
+							partRelDesc,
+							1,	/* dummy */
+							0);
+
+		ExecOpenIndices(partition, false);
+	}
+	else
+	{
+		char	   *val_desc;
+		Bitmapset  *modifiedCols;
+		Bitmapset  *insertedCols;
+		Bitmapset  *updatedCols;
+
+		insertedCols = GetInsertedColumns(resultRelInfo, estate);
+		updatedCols = GetUpdatedColumns(resultRelInfo, estate);
+		modifiedCols = bms_union(insertedCols, updatedCols);
+		val_desc = ExecBuildSlotValueDescription(RelationGetRelid(parentRelDesc),
+												 slot,
+												 RelationGetDescr(parentRelDesc),
+												 modifiedCols,
+												 64);
+
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("cannot find partition for the new row for relation \"%s\"",
+						RelationGetRelationName(parentRelDesc)),
+				 val_desc ? errdetail("Failing row contains %s.", val_desc) : 0));
+	}
+
+	return partition;
+}
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 1ef76d0..f1cc8ed 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -226,6 +226,8 @@ ExecInsert(ModifyTableState *mtstate,
 {
 	HeapTuple	tuple;
 	ResultRelInfo *resultRelInfo;
+	ResultRelInfo *saved_resultRelInfo = NULL;	/* for partitioned table case */
+	ExprContext   *econtext;
 	Relation	resultRelationDesc;
 	Oid			newId;
 	List	   *recheckIndexes = NIL;
@@ -334,6 +336,20 @@ ExecInsert(ModifyTableState *mtstate,
 		if (resultRelationDesc->rd_att->constr)
 			ExecConstraints(resultRelInfo, slot, estate);
 
+		/*
+		 * Switch resultRelInfo to one corresponding to the partition
+		 * that this tuple maps to.
+		 */
+		if (resultRelInfo->ri_Partitions)
+		{
+			saved_resultRelInfo = resultRelInfo;
+			/* for partition key expressions */
+			econtext = GetPerTupleExprContext(estate);
+			econtext->ecxt_scantuple = slot;
+			resultRelInfo = ExecFindPartition(resultRelInfo, slot, estate);
+			estate->es_result_relation_info = resultRelInfo;
+		}
+
 		if (onconflict != ONCONFLICT_NONE && resultRelInfo->ri_NumIndices > 0)
 		{
 			/* Perform a speculative insertion. */
@@ -449,7 +465,7 @@ ExecInsert(ModifyTableState *mtstate,
 			 * Note: heap_insert returns the tid (location) of the new tuple
 			 * in the t_self field.
 			 */
-			newId = heap_insert(resultRelationDesc, tuple,
+			newId = heap_insert(resultRelInfo->ri_RelationDesc, tuple,
 								estate->es_output_cid,
 								0, NULL);
 
@@ -471,6 +487,17 @@ ExecInsert(ModifyTableState *mtstate,
 	/* AFTER ROW INSERT Triggers */
 	ExecARInsertTriggers(estate, resultRelInfo, tuple, recheckIndexes);
 
+	/* Restore the resultRelInfo, if had been switched. */
+	if (saved_resultRelInfo)
+	{
+		heap_close(resultRelInfo->ri_RelationDesc, RowExclusiveLock);
+		ExecCloseIndices(resultRelInfo);
+		pfree(resultRelInfo);
+		resultRelInfo = saved_resultRelInfo;
+		estate->es_result_relation_info = resultRelInfo;
+		saved_resultRelInfo = NULL; /* aka, paranoia! */
+	}
+
 	list_free(recheckIndexes);
 
 	/*
@@ -1364,7 +1391,9 @@ ExecModifyTable(ModifyTableState *node)
 				bool		isNull;
 
 				relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind;
-				if (relkind == RELKIND_RELATION || relkind == RELKIND_MATVIEW)
+				if (relkind == RELKIND_RELATION ||
+					relkind == RELKIND_PARTITIONED_REL ||
+					relkind == RELKIND_MATVIEW)
 				{
 					datum = ExecGetJunkAttribute(slot,
 												 junkfilter->jf_junkAttNo,
@@ -1800,6 +1829,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 					relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind;
 					if (relkind == RELKIND_RELATION ||
+						relkind == RELKIND_PARTITIONED_REL ||
 						relkind == RELKIND_MATVIEW)
 					{
 						j->jf_junkAttNo = ExecFindJunkAttribute(j, "ctid");
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 1c8425d..cee3473 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2593,6 +2593,57 @@ _copyColumnDef(const ColumnDef *from)
 	return newnode;
 }
 
+static PartitionBy *
+_copyPartitionBy(const PartitionBy *from)
+{
+
+	PartitionBy *newnode = makeNode(PartitionBy);
+
+	COPY_SCALAR_FIELD(strategy);
+	COPY_NODE_FIELD(partParams);
+	COPY_LOCATION_FIELD(location);
+
+	return newnode;
+}
+
+static PartitionElem *
+_copyPartitionElem(const PartitionElem *from)
+{
+	PartitionElem *newnode = makeNode(PartitionElem);
+
+	COPY_STRING_FIELD(name);
+	COPY_NODE_FIELD(expr);
+	COPY_NODE_FIELD(opclass);
+	COPY_LOCATION_FIELD(location);
+
+	return newnode;
+}
+
+static PartitionValues *
+_copyPartitionValues(const PartitionValues *from)
+{
+	PartitionValues *newnode = makeNode(PartitionValues);
+
+	COPY_NODE_FIELD(listvalues);
+	COPY_NODE_FIELD(rangemins);
+	COPY_NODE_FIELD(rangemaxs);
+	COPY_LOCATION_FIELD(location);
+
+	return newnode;
+}
+
+static PartitionDef *
+_copyPartitionDef(const PartitionDef *from)
+{
+	PartitionDef *newnode = makeNode(PartitionDef);
+
+	COPY_NODE_FIELD(name);
+	COPY_NODE_FIELD(parent);
+	COPY_NODE_FIELD(values);
+
+	return newnode;
+}
+
 static Constraint *
 _copyConstraint(const Constraint *from)
 {
@@ -2828,6 +2879,7 @@ _copyAlterTableCmd(const AlterTableCmd *from)
 	COPY_STRING_FIELD(name);
 	COPY_NODE_FIELD(newowner);
 	COPY_NODE_FIELD(def);
+	COPY_NODE_FIELD(using_table);
 	COPY_SCALAR_FIELD(behavior);
 	COPY_SCALAR_FIELD(missing_ok);
 
@@ -2974,8 +3026,11 @@ static void
 CopyCreateStmtFields(const CreateStmt *from, CreateStmt *newnode)
 {
 	COPY_NODE_FIELD(relation);
+	COPY_NODE_FIELD(partitionOf);
 	COPY_NODE_FIELD(tableElts);
 	COPY_NODE_FIELD(inhRelations);
+	COPY_NODE_FIELD(partValues);
+	COPY_NODE_FIELD(partitionby);
 	COPY_NODE_FIELD(ofTypename);
 	COPY_NODE_FIELD(constraints);
 	COPY_NODE_FIELD(options);
@@ -4907,6 +4962,18 @@ copyObject(const void *from)
 		case T_ColumnDef:
 			retval = _copyColumnDef(from);
 			break;
+		case T_PartitionBy:
+			retval = _copyPartitionBy(from);
+			break;
+		case T_PartitionElem:
+			retval = _copyPartitionElem(from);
+			break;
+		case T_PartitionValues:
+			retval = _copyPartitionValues(from);
+			break;
+		case T_PartitionDef:
+			retval = _copyPartitionDef(from);
+			break;
 		case T_Constraint:
 			retval = _copyConstraint(from);
 			break;
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 1d6c43c..27e9420 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -1020,6 +1020,7 @@ _equalAlterTableCmd(const AlterTableCmd *a, const AlterTableCmd *b)
 	COMPARE_STRING_FIELD(name);
 	COMPARE_NODE_FIELD(newowner);
 	COMPARE_NODE_FIELD(def);
+	COMPARE_NODE_FIELD(using_table);
 	COMPARE_SCALAR_FIELD(behavior);
 	COMPARE_SCALAR_FIELD(missing_ok);
 
@@ -1140,8 +1141,11 @@ static bool
 _equalCreateStmt(const CreateStmt *a, const CreateStmt *b)
 {
 	COMPARE_NODE_FIELD(relation);
+	COMPARE_NODE_FIELD(partitionOf);
 	COMPARE_NODE_FIELD(tableElts);
 	COMPARE_NODE_FIELD(inhRelations);
+	COMPARE_NODE_FIELD(partValues);
+	COMPARE_NODE_FIELD(partitionby);
 	COMPARE_NODE_FIELD(ofTypename);
 	COMPARE_NODE_FIELD(constraints);
 	COMPARE_NODE_FIELD(options);
@@ -2338,6 +2342,48 @@ _equalColumnDef(const ColumnDef *a, const ColumnDef *b)
 }
 
 static bool
+_equalPartitionBy(const PartitionBy *a, const PartitionBy *b)
+{
+	COMPARE_SCALAR_FIELD(strategy);
+	COMPARE_NODE_FIELD(partParams);
+	COMPARE_LOCATION_FIELD(location);
+
+	return true;
+}
+
+static bool
+_equalPartitionElem(const PartitionElem *a, const PartitionElem *b)
+{
+	COMPARE_STRING_FIELD(name);
+	COMPARE_NODE_FIELD(expr);
+	COMPARE_NODE_FIELD(opclass);
+	COMPARE_LOCATION_FIELD(location);
+
+	return true;
+}
+
+static bool
+_equalPartitionValues(const PartitionValues *a, const PartitionValues *b)
+{
+	COMPARE_NODE_FIELD(listvalues);
+	COMPARE_NODE_FIELD(rangemins);
+	COMPARE_NODE_FIELD(rangemaxs);
+	COMPARE_LOCATION_FIELD(location);
+
+	return true;
+}
+
+static bool
+_equalPartitionDef(const PartitionDef *a, const PartitionDef *b)
+{
+	COMPARE_NODE_FIELD(name);
+	COMPARE_NODE_FIELD(parent);
+	COMPARE_NODE_FIELD(values);
+
+	return true;
+}
+
+static bool
 _equalConstraint(const Constraint *a, const Constraint *b)
 {
 	COMPARE_SCALAR_FIELD(contype);
@@ -3264,6 +3310,18 @@ equal(const void *a, const void *b)
 		case T_ColumnDef:
 			retval = _equalColumnDef(a, b);
 			break;
+		case T_PartitionBy:
+			retval = _equalPartitionBy(a, b);
+			break;
+		case T_PartitionElem:
+			retval = _equalPartitionElem(a, b);
+			break;
+		case T_PartitionValues:
+			retval = _equalPartitionValues(a, b);
+			break;
+		case T_PartitionDef:
+			retval = _equalPartitionDef(a, b);
+			break;
 		case T_Constraint:
 			retval = _equalConstraint(a, b);
 			break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index a878498..ca79fba 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2092,8 +2092,11 @@ static void
 _outCreateStmtInfo(StringInfo str, const CreateStmt *node)
 {
 	WRITE_NODE_FIELD(relation);
+	WRITE_NODE_FIELD(partitionOf);
 	WRITE_NODE_FIELD(tableElts);
 	WRITE_NODE_FIELD(inhRelations);
+	WRITE_NODE_FIELD(partValues);
+	WRITE_NODE_FIELD(partitionby);
 	WRITE_NODE_FIELD(ofTypename);
 	WRITE_NODE_FIELD(constraints);
 	WRITE_NODE_FIELD(options);
@@ -2333,6 +2336,48 @@ _outIndexElem(StringInfo str, const IndexElem *node)
 }
 
 static void
+_outPartitionBy(StringInfo str, const PartitionBy *node)
+{
+	WRITE_NODE_TYPE("PARTITIONBY");
+
+	WRITE_CHAR_FIELD(strategy);
+	WRITE_NODE_FIELD(partParams);
+	WRITE_LOCATION_FIELD(location);
+}
+
+static void
+_outPartitionElem(StringInfo str, const PartitionElem *node)
+{
+	WRITE_NODE_TYPE("PARTITIONELEM");
+
+	WRITE_STRING_FIELD(name);
+	WRITE_NODE_FIELD(expr);
+	WRITE_NODE_FIELD(opclass);
+	WRITE_LOCATION_FIELD(location);
+}
+
+static void
+_outPartitionValues(StringInfo str, const PartitionValues *node)
+{
+	WRITE_NODE_TYPE("PARTITIONVALUES");
+
+	WRITE_NODE_FIELD(listvalues);
+	WRITE_NODE_FIELD(rangemins);
+	WRITE_NODE_FIELD(rangemaxs);
+	WRITE_LOCATION_FIELD(location);
+}
+
+static void
+_outPartitionDef(StringInfo str, const PartitionDef *node)
+{
+	WRITE_NODE_TYPE("PARTITIONDEF");
+
+	WRITE_NODE_FIELD(name);
+	WRITE_NODE_FIELD(parent);
+	WRITE_NODE_FIELD(values);
+}
+
+static void
 _outQuery(StringInfo str, const Query *node)
 {
 	WRITE_NODE_TYPE("QUERY");
@@ -3366,6 +3411,18 @@ _outNode(StringInfo str, const void *obj)
 			case T_IndexElem:
 				_outIndexElem(str, obj);
 				break;
+			case T_PartitionBy:
+				_outPartitionBy(str, obj);
+				break;
+			case T_PartitionElem:
+				_outPartitionElem(str, obj);
+				break;
+			case T_PartitionValues:
+				_outPartitionValues(str, obj);
+				break;
+			case T_PartitionDef:
+				_outPartitionDef(str, obj);
+				break;
 			case T_Query:
 				_outQuery(str, obj);
 				break;
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 426a09d..df52df5 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -226,6 +226,9 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	struct ImportQual	*importqual;
 	InsertStmt			*istmt;
 	VariableSetStmt		*vsetstmt;
+	PartitionElem		*pelem;
+	PartitionValues		*pvalues;
+	PartitionBy			*partby;
 }
 
 %type <node>	stmt schema_stmt
@@ -321,6 +324,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 				opt_collate
 
 %type <range>	qualified_name insert_target OptConstrFromTable
+				using_table opt_using_table
 
 %type <str>		all_Op MathOp
 
@@ -537,6 +541,13 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 				opt_frame_clause frame_extent frame_bound
 %type <str>		opt_existing_window_name
 %type <boolean> opt_if_not_exists
+%type <partby>	OptPartitionBy PartitionBy
+%type <list>	part_params
+%type <pelem> 	part_elem
+%type <pvalues>	PartitionValues
+%type <pvalues>	ListValues
+%type <pvalues>	RangeValues
+%type <list>	ValuesList
 
 /*
  * Non-keyword token types.  These are hard-wired into the "flex" lexer.
@@ -562,7 +573,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 /* ordinary key words in alphabetical order */
 %token <keyword> ABORT_P ABSOLUTE_P ACCESS ACTION ADD_P ADMIN AFTER
 	AGGREGATE ALL ALSO ALTER ALWAYS ANALYSE ANALYZE AND ANY ARRAY AS ASC
-	ASSERTION ASSIGNMENT ASYMMETRIC AT ATTRIBUTE AUTHORIZATION
+	ASSERTION ASSIGNMENT ASYMMETRIC AT ATTACH ATTRIBUTE AUTHORIZATION
 
 	BACKWARD BEFORE BEGIN_P BETWEEN BIGINT BINARY BIT
 	BOOLEAN_P BOTH BY
@@ -577,7 +588,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	CURRENT_TIME CURRENT_TIMESTAMP CURRENT_USER CURSOR CYCLE
 
 	DATA_P DATABASE DAY_P DEALLOCATE DEC DECIMAL_P DECLARE DEFAULT DEFAULTS
-	DEFERRABLE DEFERRED DEFINER DELETE_P DELIMITER DELIMITERS DESC
+	DEFERRABLE DEFERRED DEFINER DELETE_P DELIMITER DELIMITERS DESC DETACH
 	DICTIONARY DISABLE_P DISCARD DISTINCT DO DOCUMENT_P DOMAIN_P DOUBLE_P DROP
 
 	EACH ELSE ENABLE_P ENCODING ENCRYPTED END_P ENUM_P ESCAPE EVENT EXCEPT
@@ -601,7 +612,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 	KEY
 
 	LABEL LANGUAGE LARGE_P LAST_P LATERAL_P
-	LEADING LEAKPROOF LEAST LEFT LEVEL LIKE LIMIT LISTEN LOAD LOCAL
+	LEADING LEAKPROOF LEAST LEFT LEVEL LIKE LIMIT LIST LISTEN LOAD LOCAL
 	LOCALTIME LOCALTIMESTAMP LOCATION LOCK_P LOCKED LOGGED
 
 	MAPPING MATCH MATERIALIZED MAXVALUE MINUTE_P MINVALUE MODE MONTH_P MOVE
@@ -2360,6 +2371,44 @@ alter_table_cmd:
 					n->def = (Node *)$1;
 					$$ = (Node *) n;
 				}
+			/*
+			 * ALTER TABLE <parent> ATTACH PARTITION <name>
+			 *		FOR VALUES (...) USING [TABLE] <table_name>
+			 */
+			| ATTACH PARTITION qualified_name PartitionValues using_table
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionDef *def = makeNode(PartitionDef);
+					n->subtype = AT_AttachPartition;
+					def->name = $3;
+					def->values = $4;
+					n->def = (Node *) def;
+					n->using_table = $5;
+					$$ = (Node *) n;
+				}
+			/*
+			 * ALTER TABLE <parent> DETACH PARTITION <name>
+			 *		[ USING [ TABLE ] <table_name> ]
+			 */
+			| DETACH PARTITION qualified_name opt_using_table
+				{
+					AlterTableCmd *n = makeNode(AlterTableCmd);
+					PartitionDef *def = makeNode(PartitionDef);
+					n->subtype = AT_DetachPartition;
+					def->name = $3;
+					n->def = (Node *) def;
+					n->using_table = $4;
+					$$ = (Node *) n;
+				}
+		;
+
+opt_using_table:
+			using_table							{ $$ = $1; }
+			| /* EMPTY */						{ $$ = NULL; }
+		;
+using_table:
+			/* USING [ TABLE ] qualified_name */
+			USING opt_table qualified_name		{ $$ = $3; }
 		;
 
 alter_column_default:
@@ -2797,7 +2846,7 @@ copy_generic_opt_arg_list_item:
  *****************************************************************************/
 
 CreateStmt:	CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
-			OptInherit OptWith OnCommitOption OptTableSpace
+			OptInherit OptPartitionBy OptWith OnCommitOption OptTableSpace
 				{
 					CreateStmt *n = makeNode(CreateStmt);
 					$4->relpersistence = $2;
@@ -2805,15 +2854,16 @@ CreateStmt:	CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
 					n->tableElts = $6;
 					n->inhRelations = $8;
 					n->ofTypename = NULL;
+					n->partitionby = $9;
 					n->constraints = NIL;
-					n->options = $9;
-					n->oncommit = $10;
-					n->tablespacename = $11;
+					n->options = $10;
+					n->oncommit = $11;
+					n->tablespacename = $12;
 					n->if_not_exists = false;
 					$$ = (Node *)n;
 				}
 		| CREATE OptTemp TABLE IF_P NOT EXISTS qualified_name '('
-			OptTableElementList ')' OptInherit OptWith OnCommitOption
+			OptTableElementList ')' OptInherit OptPartitionBy OptWith OnCommitOption
 			OptTableSpace
 				{
 					CreateStmt *n = makeNode(CreateStmt);
@@ -2822,15 +2872,16 @@ CreateStmt:	CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
 					n->tableElts = $9;
 					n->inhRelations = $11;
 					n->ofTypename = NULL;
+					n->partitionby = $12;
 					n->constraints = NIL;
-					n->options = $12;
-					n->oncommit = $13;
-					n->tablespacename = $14;
+					n->options = $13;
+					n->oncommit = $14;
+					n->tablespacename = $15;
 					n->if_not_exists = true;
 					$$ = (Node *)n;
 				}
 		| CREATE OptTemp TABLE qualified_name OF any_name
-			OptTypedTableElementList OptWith OnCommitOption OptTableSpace
+			OptTypedTableElementList OptPartitionBy OptWith OnCommitOption OptTableSpace
 				{
 					CreateStmt *n = makeNode(CreateStmt);
 					$4->relpersistence = $2;
@@ -2839,15 +2890,16 @@ CreateStmt:	CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
 					n->inhRelations = NIL;
 					n->ofTypename = makeTypeNameFromNameList($6);
 					n->ofTypename->location = @6;
+					n->partitionby = $8;
 					n->constraints = NIL;
-					n->options = $8;
-					n->oncommit = $9;
-					n->tablespacename = $10;
+					n->options = $9;
+					n->oncommit = $10;
+					n->tablespacename = $11;
 					n->if_not_exists = false;
 					$$ = (Node *)n;
 				}
 		| CREATE OptTemp TABLE IF_P NOT EXISTS qualified_name OF any_name
-			OptTypedTableElementList OptWith OnCommitOption OptTableSpace
+			OptTypedTableElementList OptPartitionBy OptWith OnCommitOption OptTableSpace
 				{
 					CreateStmt *n = makeNode(CreateStmt);
 					$7->relpersistence = $2;
@@ -2856,10 +2908,43 @@ CreateStmt:	CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
 					n->inhRelations = NIL;
 					n->ofTypename = makeTypeNameFromNameList($9);
 					n->ofTypename->location = @9;
+					n->partitionby = $11;
 					n->constraints = NIL;
-					n->options = $11;
-					n->oncommit = $12;
-					n->tablespacename = $13;
+					n->options = $12;
+					n->oncommit = $13;
+					n->tablespacename = $14;
+					n->if_not_exists = true;
+					$$ = (Node *)n;
+				}
+		| CREATE OptTemp TABLE qualified_name PARTITION OF qualified_name
+			PartitionValues OptPartitionBy OptWith OnCommitOption OptTableSpace
+				{
+					CreateStmt *n = makeNode(CreateStmt);
+					$4->relpersistence = $2;
+					n->relation = $4;
+					n->partitionOf = $7;
+					n->partValues = $8;
+					n->partitionby = $9;
+					n->constraints = NIL;
+					n->options = $10;
+					n->oncommit = $11;
+					n->tablespacename = $12;
+					n->if_not_exists = false;
+					$$ = (Node *)n;
+				}
+		| CREATE OptTemp TABLE IF_P NOT EXISTS qualified_name PARTITION OF qualified_name
+			PartitionValues OptPartitionBy OptWith OnCommitOption OptTableSpace
+				{
+					CreateStmt *n = makeNode(CreateStmt);
+					$7->relpersistence = $2;
+					n->relation = $7;
+					n->partitionOf = $10;
+					n->partValues = $11;
+					n->partitionby = $12;
+					n->constraints = NIL;
+					n->options = $13;
+					n->oncommit = $14;
+					n->tablespacename = $15;
 					n->if_not_exists = true;
 					$$ = (Node *)n;
 				}
@@ -3402,6 +3487,132 @@ OptInherit: INHERITS '(' qualified_name_list ')'	{ $$ = $3; }
 			| /*EMPTY*/								{ $$ = NIL; }
 		;
 
+/* Optional partition key (PARTITION ON) definition */
+OptPartitionBy: PartitionBy	{ $$ = $1; }
+			| /*EMPTY*/			{ $$ = NULL; }
+		;
+
+PartitionBy: PARTITION BY RANGE ON '(' part_params ')'
+				{
+					PartitionBy *n = makeNode(PartitionBy);
+
+					n->strategy = PARTITION_STRAT_RANGE;
+					n->partParams = $6;
+					n->location = @1;
+
+					$$ = n;
+				}
+			| PARTITION BY LIST ON '(' part_params ')'
+				{
+					PartitionBy *n = makeNode(PartitionBy);
+
+					n->strategy = PARTITION_STRAT_LIST;
+					n->partParams = $6;
+					n->location = @1;
+
+					$$ = n;
+				}
+		;
+
+part_params:	part_elem						{ $$ = list_make1($1); }
+			| part_params ',' part_elem			{ $$ = lappend($1, $3); }
+		;
+
+part_elem: ColId opt_class
+				{
+					PartitionElem *n = makeNode(PartitionElem);
+
+					n->name = $1;
+					n->expr = NULL;
+					n->opclass = $2;
+					n->location = @1;
+					$$ = n;
+				}
+			| func_expr_windowless opt_class
+				{
+					PartitionElem *n = makeNode(PartitionElem);
+
+					n->name = NULL;
+					n->expr = $1;
+					n->opclass = $2;
+					n->location = @1;
+					$$ = n;
+				}
+			| '(' a_expr ')' opt_class
+				{
+					PartitionElem *n = makeNode(PartitionElem);
+
+					n->name = NULL;
+					n->expr = $2;
+					n->opclass = $4;
+					n->location = @1;
+					$$ = n;
+				}
+		;
+
+/* Definition of a partition */
+PartitionValues:
+				FOR VALUES ListValues		{ $$ = $3; }
+			|	FOR VALUES RangeValues		{ $$ = $3; }
+		;
+
+ListValues:
+			opt_in '(' ValuesList ')'
+				{
+					PartitionValues *n = makeNode(PartitionValues);
+
+					n->listvalues = $3;
+					n->rangemins = NIL;
+					n->rangemaxs = NIL;
+					n->location = @1;
+					$$ = n;
+				}
+		;
+
+RangeValues:
+			END_P '(' ValuesList ')'
+				{
+					PartitionValues *n = makeNode(PartitionValues);
+
+					n->listvalues = NIL;
+					n->rangemins = NIL;
+					n->rangemaxs = $3;
+					n->location = @1;
+					$$ = n;
+				}
+			|
+			START '(' ValuesList ')'
+				{
+					PartitionValues *n = makeNode(PartitionValues);
+
+					n->listvalues = NIL;
+					n->rangemins = $3;
+					n->rangemaxs = NIL;
+					n->location = @1;
+					$$ = n;
+				}
+			|
+			START '(' ValuesList ')' END_P '(' ValuesList ')'
+				{
+					PartitionValues *n = makeNode(PartitionValues);
+
+					n->listvalues = NIL;
+					n->rangemins = $3;
+					n->rangemaxs = $7;
+					n->location = @1;
+					$$ = n;
+				}
+		;
+
+opt_in:		IN_P						{}
+			|	/* EMPTY */				{}
+		;
+
+ValuesList:
+			a_expr						{ $$ = list_make1($1); }
+			| ValuesList ',' a_expr		{ $$ = lappend($1, $3); }
+		;
+
 /* WITH (options) is preferred, WITH OIDS and WITHOUT OIDS are legacy forms */
 OptWith:
 			WITH reloptions				{ $$ = $2; }
@@ -13637,6 +13848,7 @@ unreserved_keyword:
 			| ASSERTION
 			| ASSIGNMENT
 			| AT
+			| ATTACH
 			| ATTRIBUTE
 			| BACKWARD
 			| BEFORE
@@ -13682,6 +13894,7 @@ unreserved_keyword:
 			| DELETE_P
 			| DELIMITER
 			| DELIMITERS
+			| DETACH
 			| DICTIONARY
 			| DISABLE_P
 			| DISCARD
@@ -13743,6 +13956,7 @@ unreserved_keyword:
 			| LAST_P
 			| LEAKPROOF
 			| LEVEL
+			| LIST
 			| LISTEN
 			| LOAD
 			| LOCAL
diff --git a/src/backend/parser/parse_agg.c b/src/backend/parser/parse_agg.c
index 5b0d568..b66aac8 100644
--- a/src/backend/parser/parse_agg.c
+++ b/src/backend/parser/parse_agg.c
@@ -450,6 +450,9 @@ check_agglevels_and_constraints(ParseState *pstate, Node *expr)
 				err = _("grouping operations are not allowed in DEFAULT expressions");
 
 			break;
+		case EXPR_KIND_PARTITION_VALUES:
+			err = _("aggregate functions are not allowed in partition value expressions");
+			break;
 		case EXPR_KIND_INDEX_EXPRESSION:
 			if (isAgg)
 				err = _("aggregate functions are not allowed in index expressions");
@@ -457,6 +460,9 @@ check_agglevels_and_constraints(ParseState *pstate, Node *expr)
 				err = _("grouping operations are not allowed in index expressions");
 
 			break;
+		case EXPR_KIND_PARTKEY_EXPRESSION:
+			err = _("aggregate functions are not allowed in partition key expressions");
+			break;
 		case EXPR_KIND_INDEX_PREDICATE:
 			if (isAgg)
 				err = _("aggregate functions are not allowed in index predicates");
@@ -827,9 +833,15 @@ transformWindowFuncCall(ParseState *pstate, WindowFunc *wfunc,
 		case EXPR_KIND_FUNCTION_DEFAULT:
 			err = _("window functions are not allowed in DEFAULT expressions");
 			break;
+		case EXPR_KIND_PARTITION_VALUES:
+			err = _("window functions are not allowed in partition value expressions");
+			break;
 		case EXPR_KIND_INDEX_EXPRESSION:
 			err = _("window functions are not allowed in index expressions");
 			break;
+		case EXPR_KIND_PARTKEY_EXPRESSION:
+			err = _("window functions are not allowed in partition key expressions");
+			break;
 		case EXPR_KIND_INDEX_PREDICATE:
 			err = _("window functions are not allowed in index predicates");
 			break;
diff --git a/src/backend/parser/parse_clause.c b/src/backend/parser/parse_clause.c
index 5980856..c856215 100644
--- a/src/backend/parser/parse_clause.c
+++ b/src/backend/parser/parse_clause.c
@@ -22,6 +22,7 @@
 #include "catalog/catalog.h"
 #include "catalog/heap.h"
 #include "catalog/pg_constraint.h"
+#include "catalog/pg_partition_fn.h"
 #include "catalog/pg_type.h"
 #include "commands/defrem.h"
 #include "nodes/makefuncs.h"
@@ -2880,6 +2881,17 @@ transformOnConflictArbiter(ParseState *pstate,
 				 parser_errposition(pstate,
 								  exprLocation((Node *) onConflictClause))));
 
+	/*
+	 * ON CONFLICT on partitioned tables disallowed.
+	 * XXX - not permanently.
+	 */
+	if (IsPartitionedRel(pstate->p_target_relation))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+			  errmsg("ON CONFLICT not supported with partitioned tables"),
+				 parser_errposition(pstate,
+								  exprLocation((Node *) onConflictClause))));
+
 	/* ON CONFLICT DO NOTHING does not require an inference clause */
 	if (infer)
 	{
diff --git a/src/backend/parser/parse_expr.c b/src/backend/parser/parse_expr.c
index fa77ef1..183ae6b 100644
--- a/src/backend/parser/parse_expr.c
+++ b/src/backend/parser/parse_expr.c
@@ -1700,9 +1700,15 @@ transformSubLink(ParseState *pstate, SubLink *sublink)
 		case EXPR_KIND_FUNCTION_DEFAULT:
 			err = _("cannot use subquery in DEFAULT expression");
 			break;
+		case EXPR_KIND_PARTITION_VALUES:
+			err = _("cannot use subquery in partition value expressions");
+			break;
 		case EXPR_KIND_INDEX_EXPRESSION:
 			err = _("cannot use subquery in index expression");
 			break;
+		case EXPR_KIND_PARTKEY_EXPRESSION:
+			err = _("cannot use subquery in partition key expressions");
+			break;
 		case EXPR_KIND_INDEX_PREDICATE:
 			err = _("cannot use subquery in index predicate");
 			break;
@@ -3225,6 +3231,10 @@ ParseExprKindName(ParseExprKind exprKind)
 			return "EXECUTE";
 		case EXPR_KIND_TRIGGER_WHEN:
 			return "WHEN";
+		case EXPR_KIND_PARTKEY_EXPRESSION:
+			return "partition key expression";
+		case EXPR_KIND_PARTITION_VALUES:
+			return "PARTITION FOR VALUES";
 
 			/*
 			 * There is intentionally no default: case here, so that the
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 16d40c7..596b550 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -83,6 +83,8 @@ typedef struct
 	List	   *alist;			/* "after list" of things to do after creating
 								 * the table */
 	IndexStmt  *pkey;			/* PRIMARY KEY index, if any */
+	bool		ispartitioned;	/* true if CREATE TABLE ... PARTITION BY */
+	List	   *partitionElts;	/* CREATE TABLE ... PARTITION OF ... */
 } CreateStmtContext;
 
 /* State shared by transformCreateSchemaStmt and its subroutines */
@@ -123,6 +125,11 @@ static void transformConstraintAttrs(CreateStmtContext *cxt,
 						 List *constraintList);
 static void transformColumnType(CreateStmtContext *cxt, ColumnDef *column);
 static void setSchemaName(char *context_schema, char **stmt_schema_name);
+static void transformPartitionOf(CreateStmtContext *cxt,
+					 RangeVar *parent,
+					 PartitionValues *values);
+static PartitionValues* transformPartitionValues(CreateStmtContext *cxt,
+								PartitionValues *values);
 
 
 /*
@@ -222,6 +229,8 @@ transformCreateStmt(CreateStmt *stmt, const char *queryString)
 	cxt.blist = NIL;
 	cxt.alist = NIL;
 	cxt.pkey = NULL;
+	cxt.ispartitioned = stmt->partitionby ? true : false;
+	cxt.partitionElts = NIL;
 
 	/*
 	 * Notice that we allow OIDs here only for plain tables, even though
@@ -236,11 +245,29 @@ transformCreateStmt(CreateStmt *stmt, const char *queryString)
 	cxt.hasoids = interpretOidsOption(stmt->options, !cxt.isforeign);
 
 	Assert(!stmt->ofTypename || !stmt->inhRelations);	/* grammar enforces */
+	Assert(!stmt->ofTypename || !stmt->partitionOf);	/* grammar enforces */
+	Assert(!stmt->inhRelations || !stmt->partitionOf);	/* grammar enforces */
 
 	if (stmt->ofTypename)
 		transformOfType(&cxt, stmt->ofTypename);
 
 	/*
+	 * Transform "PARTITION OF parent FOR VALUES ..." clause:
+	 *
+	 * 1. Gin up a (LIKE parent INCLUDING ALL) clause,
+	 * 2. Gin up a AlterTableStmt that arranges to attaches this relation
+	 * as a partition of parent.
+	 */
+	if (stmt->partitionOf)
+	{
+		transformPartitionOf(&cxt, stmt->partitionOf, stmt->partValues);
+
+		/* Override based on the result of transformPartitionOf. */
+		stmt->tableElts = cxt.partitionElts;
+		stmt->relation->schemaname = cxt.relation->schemaname;
+	}
+
+	/*
 	 * Run through each primary element in the table creation clause. Separate
 	 * column defs from constraints, and do preliminary analysis.
 	 */
@@ -541,6 +568,12 @@ transformColumnDefinition(CreateStmtContext *cxt, ColumnDef *column)
 							 errmsg("primary key constraints are not supported on foreign tables"),
 							 parser_errposition(cxt->pstate,
 												constraint->location)));
+				if (cxt->ispartitioned)
+					ereport(ERROR,
+							(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+							 errmsg("primary key constraints are not supported on partitioned tables"),
+							 parser_errposition(cxt->pstate,
+												constraint->location)));
 				/* FALL THRU */
 
 			case CONSTR_UNIQUE:
@@ -550,6 +583,12 @@ transformColumnDefinition(CreateStmtContext *cxt, ColumnDef *column)
 							 errmsg("unique constraints are not supported on foreign tables"),
 							 parser_errposition(cxt->pstate,
 												constraint->location)));
+				if (cxt->ispartitioned)
+					ereport(ERROR,
+							(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+							 errmsg("unique constraints are not supported on partitioned tables"),
+							 parser_errposition(cxt->pstate,
+												constraint->location)));
 				if (constraint->keys == NIL)
 					constraint->keys = list_make1(makeString(column->colname));
 				cxt->ixconstraints = lappend(cxt->ixconstraints, constraint);
@@ -567,6 +606,16 @@ transformColumnDefinition(CreateStmtContext *cxt, ColumnDef *column)
 							 errmsg("foreign key constraints are not supported on foreign tables"),
 							 parser_errposition(cxt->pstate,
 												constraint->location)));
+				/*
+				 * Following limitation stems from the fact that row-level AFTER
+				 * triggers are yet disallowed. See CreateTrigger().
+				 */
+				if (cxt->ispartitioned)
+					ereport(ERROR,
+							(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+							 errmsg("foreign key constraints are not supported on partitioned tables"),
+							 parser_errposition(cxt->pstate,
+												constraint->location)));
 
 				/*
 				 * Fill in the current attribute's name and throw it into the
@@ -632,6 +681,12 @@ transformTableConstraint(CreateStmtContext *cxt, Constraint *constraint)
 						 errmsg("primary key constraints are not supported on foreign tables"),
 						 parser_errposition(cxt->pstate,
 											constraint->location)));
+			if (cxt->ispartitioned)
+				ereport(ERROR,
+						(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+						 errmsg("primary key constraints are not supported on partitioned tables"),
+						 parser_errposition(cxt->pstate,
+											constraint->location)));
 			cxt->ixconstraints = lappend(cxt->ixconstraints, constraint);
 			break;
 
@@ -642,6 +697,12 @@ transformTableConstraint(CreateStmtContext *cxt, Constraint *constraint)
 						 errmsg("unique constraints are not supported on foreign tables"),
 						 parser_errposition(cxt->pstate,
 											constraint->location)));
+			if (cxt->ispartitioned)
+				ereport(ERROR,
+						(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+						 errmsg("unique constraints are not supported on partitioned tables"),
+						 parser_errposition(cxt->pstate,
+											constraint->location)));
 			cxt->ixconstraints = lappend(cxt->ixconstraints, constraint);
 			break;
 
@@ -652,6 +713,12 @@ transformTableConstraint(CreateStmtContext *cxt, Constraint *constraint)
 						 errmsg("exclusion constraints are not supported on foreign tables"),
 						 parser_errposition(cxt->pstate,
 											constraint->location)));
+			if (cxt->ispartitioned)
+				ereport(ERROR,
+						(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+						 errmsg("exclusion constraints are not supported on partitioned tables"),
+						 parser_errposition(cxt->pstate,
+											constraint->location)));
 			cxt->ixconstraints = lappend(cxt->ixconstraints, constraint);
 			break;
 
@@ -666,6 +733,16 @@ transformTableConstraint(CreateStmtContext *cxt, Constraint *constraint)
 						 errmsg("foreign key constraints are not supported on foreign tables"),
 						 parser_errposition(cxt->pstate,
 											constraint->location)));
+			/*
+			 * Following limitation stems from the fact that row-level AFTER
+			 * triggers are yet disallowed. See CreateTrigger().
+			 */
+			if (cxt->ispartitioned)
+				ereport(ERROR,
+						(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+						 errmsg("foreign key constraints are not supported on partitioned tables"),
+						 parser_errposition(cxt->pstate,
+											constraint->location)));
 			cxt->fkconstraints = lappend(cxt->fkconstraints, constraint);
 			break;
 
@@ -718,6 +795,7 @@ transformTableLikeClause(CreateStmtContext *cxt, TableLikeClause *table_like_cla
 	relation = relation_openrv(table_like_clause->relation, AccessShareLock);
 
 	if (relation->rd_rel->relkind != RELKIND_RELATION &&
+		relation->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 		relation->rd_rel->relkind != RELKIND_VIEW &&
 		relation->rd_rel->relkind != RELKIND_MATVIEW &&
 		relation->rd_rel->relkind != RELKIND_COMPOSITE_TYPE &&
@@ -2430,6 +2508,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
 	cxt.inh_indexes = NIL;
 	cxt.blist = NIL;
 	cxt.alist = NIL;
+	cxt.ispartitioned = is_partitioned(relid) ? true : false;
 	cxt.pkey = NULL;
 
 	/*
@@ -2519,6 +2598,52 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
 					break;
 				}
 
+			/*
+			 * Push a RenameStmt to rename the the table to specified name
+			 * once it's attached to become a partition. Rest of the
+			 * responsibilities belong to AlterTblStmt.
+			 *
+			 * XXX - This is currently not supported. Won't be until we teach
+			 * ATExecAttachPartition() to deal with existing tables passed in
+			 * to use as a partition. Remember to set cmd->def->parent:
+			 */
+			case AT_AttachPartition:
+				{
+					if (cmd->using_table)
+					{
+						ereport(ERROR,
+								(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+								 errmsg("ALTER TABLE ATTACH PARTITION (...) USING TABLE not implemented")));
+					}
+
+					newcmds = lappend(newcmds, cmd);
+					break;
+				}
+
+			/*
+			 * Push a RenameStmt to rename the partition to specified name
+			 * once it's detached to become a regular table.
+			 */
+			case AT_DetachPartition:
+				{
+					RenameStmt	*stmt;
+
+					if (cmd->using_table)
+					{
+						stmt = makeNode(RenameStmt);
+
+						stmt->renameType = OBJECT_TABLE;
+						stmt->relation = ((PartitionDef *) cmd->def)->name;
+						stmt->newname = cmd->using_table->relname;
+						stmt->missing_ok = false;
+
+						cxt.alist = lappend(cxt.alist, stmt);
+					}
+
+					newcmds = lappend(newcmds, cmd);
+					break;
+				}
+
 			default:
 				newcmds = lappend(newcmds, cmd);
 				break;
@@ -2883,3 +3008,105 @@ setSchemaName(char *context_schema, char **stmt_schema_name)
 						"different from the one being created (%s)",
 						*stmt_schema_name, context_schema)));
 }
+
+/*
+ * transformPartitionOf - transform CREATE TABLE ... PARTITION OF <parent>
+ */
+static void
+transformPartitionOf(CreateStmtContext *cxt,
+					 RangeVar *parent,
+					 PartitionValues *values)
+{
+	TableLikeClause *like;
+	PartitionDef	*partition;
+	AlterTableCmd	*attachcmd;
+	AlterTableStmt	*alter;
+
+	/* Use the same schema as the parent if not specified. */
+	if (cxt->relation->schemaname == NULL)
+		cxt->relation->schemaname = parent->schemaname;
+
+	like = makeNode(TableLikeClause);
+	like->relation = parent;
+	like->options = CREATE_TABLE_LIKE_ALL;
+
+	cxt->partitionElts = list_make1(like);
+
+	/*
+	 * Gin up a AlterTableStmt that would take care of creating the
+	 * pg_partition entry for this relation as a partition of parent.
+	 */
+	partition = makeNode(PartitionDef);
+	partition->name = cxt->relation;
+	partition->parent = parent;
+	partition->values = transformPartitionValues(cxt, values);
+
+	attachcmd = makeNode(AlterTableCmd);
+	attachcmd->subtype = AT_AttachPartition;
+	attachcmd->def = (Node *) partition;
+
+	alter = makeNode(AlterTableStmt);
+	alter->relation = parent;
+	alter->cmds = list_make1(attachcmd);
+	alter->relkind = OBJECT_TABLE;
+
+	cxt->alist = lappend(cxt->alist, alter);
+}
+
+/*
+ * transformPartitionValues
+ *
+ * Transform partition value as returned by the grammar into something
+ * we can evaluate to store into pg_partition.
+ */
+static PartitionValues *
+transformPartitionValues(CreateStmtContext *cxt, PartitionValues *values)
+{
+	ListCell		   *cell;
+	PartitionValues	   *result = (PartitionValues *)
+											makeNode(PartitionValues);
+
+	if (values->listvalues)
+	{
+		foreach(cell, values->listvalues)
+		{
+			Node *value = (Node *) lfirst(cell);
+
+			result->listvalues = lappend(result->listvalues,
+									transformExpr(cxt->pstate, value,
+										EXPR_KIND_PARTITION_VALUES));
+		}
+	}
+	else
+	{
+		if(values->rangemins)
+		{
+			foreach(cell, values->rangemins)
+			{
+				Node *value;
+
+				value = (Node *) lfirst(cell);
+				result->rangemins = lappend(result->rangemins,
+										transformExpr(cxt->pstate, value,
+												EXPR_KIND_PARTITION_VALUES));
+			}
+		}
+
+		if(values->rangemaxs)
+		{
+			foreach(cell, values->rangemaxs)
+			{
+				Node *value;
+
+				value = (Node *) lfirst(cell);
+				result->rangemaxs = lappend(result->rangemaxs,
+										transformExpr(cxt->pstate, value,
+												EXPR_KIND_PARTITION_VALUES));
+			}
+		}
+	}
+
+	result->location = values->location;
+
+	return result;
+}
diff --git a/src/backend/rewrite/rewriteDefine.c b/src/backend/rewrite/rewriteDefine.c
index 39c83a6..b6c6d96 100644
--- a/src/backend/rewrite/rewriteDefine.c
+++ b/src/backend/rewrite/rewriteDefine.c
@@ -260,6 +260,7 @@ DefineQueryRewrite(char *rulename,
 	 * blocks them for users.  Don't mention them in the error message.
 	 */
 	if (event_relation->rd_rel->relkind != RELKIND_RELATION &&
+		event_relation->rd_rel->relkind != RELKIND_PARTITIONED_REL &&
 		event_relation->rd_rel->relkind != RELKIND_MATVIEW &&
 		event_relation->rd_rel->relkind != RELKIND_VIEW)
 		ereport(ERROR,
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index e81bbc6..63b555e 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -961,37 +961,44 @@ ProcessUtilitySlow(Node *parsetree,
 						{
 							Datum		toast_options;
 							static char *validnsps[] = HEAP_RELOPT_NAMESPACES;
+							char		relkind = ((CreateStmt *) stmt)->partitionby
+													? RELKIND_PARTITIONED_REL
+													: RELKIND_RELATION;
 
 							/* Create the table itself */
 							address = DefineRelation((CreateStmt *) stmt,
-													 RELKIND_RELATION,
+													 relkind,
 													 InvalidOid, NULL);
 							EventTriggerCollectSimpleCommand(address,
 															 secondaryObject,
 															 stmt);
 
-							/*
-							 * Let NewRelationCreateToastTable decide if this
-							 * one needs a secondary relation too.
-							 */
-							CommandCounterIncrement();
-
-							/*
-							 * parse and validate reloptions for the toast
-							 * table
-							 */
-							toast_options = transformRelOptions((Datum) 0,
-											  ((CreateStmt *) stmt)->options,
-																"toast",
-																validnsps,
-																true,
-																false);
-							(void) heap_reloptions(RELKIND_TOASTVALUE,
-												   toast_options,
-												   true);
+							/* XXX - is this the right place? */
+							if (relkind != RELKIND_PARTITIONED_REL)
+							{
+								/*
+								 * Let NewRelationCreateToastTable decide if this
+								 * one needs a secondary relation too.
+								 */
+								CommandCounterIncrement();
 
-							NewRelationCreateToastTable(address.objectId,
-														toast_options);
+								/*
+								 * parse and validate reloptions for the toast
+								 * table
+								 */
+								toast_options = transformRelOptions((Datum) 0,
+												  ((CreateStmt *) stmt)->options,
+																	"toast",
+																	validnsps,
+																	true,
+																	false);
+								(void) heap_reloptions(RELKIND_TOASTVALUE,
+													   toast_options,
+													   true);
+
+								NewRelationCreateToastTable(address.objectId,
+															toast_options);
+							}
 						}
 						else if (IsA(stmt, CreateForeignTableStmt))
 						{
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 44e9509..f8712b5 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -30,6 +30,7 @@
 #include <fcntl.h>
 #include <unistd.h>
 
+#include "access/nbtree.h"
 #include "access/htup_details.h"
 #include "access/multixact.h"
 #include "access/reloptions.h"
@@ -39,6 +40,7 @@
 #include "access/xlog.h"
 #include "catalog/catalog.h"
 #include "catalog/index.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
 #include "catalog/pg_amproc.h"
@@ -46,9 +48,13 @@
 #include "catalog/pg_authid.h"
 #include "catalog/pg_auth_members.h"
 #include "catalog/pg_constraint.h"
+#include "catalog/pg_collation.h"
 #include "catalog/pg_database.h"
 #include "catalog/pg_namespace.h"
 #include "catalog/pg_opclass.h"
+#include "catalog/pg_partition.h"
+#include "catalog/pg_partition_fn.h"
+#include "catalog/pg_partitioned_rel.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_rewrite.h"
 #include "catalog/pg_tablespace.h"
@@ -59,6 +65,9 @@
 #include "commands/policy.h"
 #include "commands/trigger.h"
 #include "miscadmin.h"
+#include "nodes/nodeFuncs.h"
+#include "nodes/execnodes.h"
+#include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
 #include "optimizer/prep.h"
@@ -69,10 +78,12 @@
 #include "storage/smgr.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
+#include "utils/datum.h"
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
+#include "utils/partition.h"
 #include "utils/relmapper.h"
 #include "utils/resowner_private.h"
 #include "utils/snapmgr.h"
@@ -275,6 +286,9 @@ static OpClassCacheEnt *LookupOpclassInfo(Oid operatorClassOid,
 				  StrategyNumber numSupport);
 static void RelationCacheInitFileRemoveInDir(const char *tblspcpath);
 static void unlink_initfile(const char *initfilename);
+static void RelationBuildPartitionKey(Relation relation);
+static PartitionInfo *CopyPartitionInfo(PartitionInfo *src, int partnatts,
+						PartitionKeyTypeInfo *typinfo);
 
 
 /*
@@ -1048,6 +1062,10 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
 	else
 		relation->rd_rsdesc = NULL;
 
+	/* if a partitioned table - initialize the key info */
+	if (relation->rd_rel->relkind == RELKIND_PARTITIONED_REL)
+		RelationBuildPartitionKey(relation);
+
 	/*
 	 * if it's an index, initialize index-related information
 	 */
@@ -2007,6 +2025,12 @@ RelationDestroyRelation(Relation relation, bool remember_tupdesc)
 	bms_free(relation->rd_keyattr);
 	bms_free(relation->rd_idattr);
 	FreeTriggerDesc(relation->trigdesc);
+	if (relation->rd_pkeytuple)
+		pfree(relation->rd_pkeytuple);
+	if (relation->rd_pkeycxt)
+		MemoryContextDelete(relation->rd_pkeycxt);
+	if (relation->rd_partlistcxt)
+		MemoryContextDelete(relation->rd_partlistcxt);
 	if (relation->rd_options)
 		pfree(relation->rd_options);
 	if (relation->rd_indextuple)
@@ -4445,6 +4469,471 @@ RelationGetExclusionInfo(Relation indexRelation,
 	MemoryContextSwitchTo(oldcxt);
 }
 
+/*
+ * RelationGetPartitionExpressions -- get the partition expressions for a
+ * partitioned table
+ *
+ * We cache the result of transforming pg_partitioned_rel.partexprs into
+ * a node tree. If the key has no expressional columns, we return NIL.
+ * Otherwise, the returned tree is copied into the caller's memory context.
+ * (We don't want to return a pointer to the relcache copy,since it could
+ * disappear due to relcache invalidation.)
+ */
+List *
+RelationGetPartitionExpressions(Relation relation)
+{
+	Relation	pg_partitioned_rel;
+	List	   *result;
+	Datum		exprsDatum;
+	bool		isnull;
+	char	   *exprsString;
+	MemoryContext oldcxt;
+
+	/* Quick exit if we already computed the result. */
+	if (relation->rd_partexprs)
+		return (List *) copyObject(relation->rd_partexprs);
+
+	/*
+	 * Quick exit if there is nothing to do.
+	 *
+	 * The first part should not happen really. We wouldn't be calling
+	 * this on a non-RELKIND_PARTITIONED_REL relation.
+	 */
+	if (relation->rd_pkeytuple == NULL ||
+		heap_attisnull(relation->rd_pkeytuple, Anum_pg_partitioned_rel_partexprs))
+		return NIL;
+
+	/*
+	 * We build the tree we intend to return in the caller's context. After
+	 * successfully completing the work, we copy it into the relcache entry.
+	 * This avoids problems if we get some sort of error partway through.
+	 */
+	pg_partitioned_rel = heap_open(PartitionedRelRelationId, AccessShareLock);
+	exprsDatum = heap_getattr(relation->rd_pkeytuple,
+							Anum_pg_partitioned_rel_partexprs,
+							RelationGetDescr(pg_partitioned_rel),
+							&isnull);
+
+	Assert(!isnull);
+	exprsString = TextDatumGetCString(exprsDatum);
+	result = (List *) stringToNode(exprsString);
+	pfree(exprsString);
+	heap_close(pg_partitioned_rel, AccessShareLock);
+
+	/*
+	 * Run the expressions through eval_const_expressions. This is not just an
+	 * optimization, but is necessary, because the planner will be comparing
+	 * them to similarly-processed qual clauses, and may fail to detect valid
+	 * matches without this.  We don't bother with canonicalize_qual, however.
+	 */
+	result = (List *) eval_const_expressions(NULL, (Node *) result);
+
+	/* May as well fix opfuncids too */
+	fix_opfuncids((Node *) result);
+
+	/* Now save a copy of the completed tree in the relcache entry. */
+	oldcxt = MemoryContextSwitchTo(relation->rd_pkeycxt);
+	relation->rd_partexprs = (List *) copyObject(result);
+	MemoryContextSwitchTo(oldcxt);
+
+	return result;
+}
+
+/*
+ * CopyPartitionInfo - copies the PartitionInfo of a partitioned relation
+ * into caller's set context
+ */
+static PartitionInfo *
+CopyPartitionInfo(PartitionInfo *src, int partnatts,
+								PartitionKeyTypeInfo *typinfo)
+{
+	PartitionInfo  *result;
+	int				i, j;
+
+	if (src == NULL)
+		return NULL;
+
+	result = (PartitionInfo *) palloc0(sizeof(PartitionInfo));
+
+	result->numpartitions = src->numpartitions;
+	result->strategy = src->strategy;
+	result->oids = (Oid *) palloc0(src->numpartitions * sizeof(Oid));
+
+	switch (src->strategy)
+	{
+		case PARTITION_STRAT_LIST:
+			result->listnvalues = (int *) palloc0(src->numpartitions
+													* sizeof(int));
+			result->listvalues = (Datum **) palloc0(src->numpartitions
+													* sizeof(Datum *));
+			for (i = 0; i < src->numpartitions; i++)
+			{
+				result->oids[i] = src->oids[i];
+				result->listnvalues[i] = src->listnvalues[i];
+				result->listvalues[i] = (Datum *)
+							palloc0(result->listnvalues[i] * sizeof(Datum));
+
+				for (j = 0; j < result->listnvalues[i]; j++)
+				{
+					Datum   *from = &src->listvalues[i][j];
+					Datum   *into = &result->listvalues[i][j];
+
+					if (!typinfo->typbyval[0])
+					{
+						if (typinfo->typlen[0] == -1)
+							*into = PointerGetDatum(PG_DETOAST_DATUM_COPY(*from));
+						else
+							*into = datumCopy(*from, false, typinfo->typlen[0]);
+					}
+					else
+						*into = *from;
+				}
+			}
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			result->rangeminnull = (bool *) palloc0(src->numpartitions
+													* sizeof(bool));
+			result->rangemaxnull = (bool *) palloc0(src->numpartitions
+													* sizeof(bool));
+			for (i = 0; i < partnatts; i++)
+			{
+				result->rangemins[i] = (Datum *) palloc0(src->numpartitions
+													* sizeof(Datum));
+				result->rangemaxs[i] = (Datum *) palloc0(src->numpartitions
+													* sizeof(Datum));
+			}
+
+			for (i = 0; i < src->numpartitions; i++)
+			{
+				result->oids[i] = src->oids[i];
+				result->rangeminnull[i] = src->rangeminnull[i];
+				result->rangemaxnull[i] = src->rangemaxnull[i];
+
+				for (j = 0; j < partnatts; j++)
+				{
+					if (!result->rangeminnull[i])
+					{
+						Datum   *from = &src->rangemins[j][i];
+						Datum   *into = &result->rangemins[j][i];
+
+						if (!typinfo->typbyval[j])
+						{
+							if (typinfo->typlen[j] == -1)
+								*into = PointerGetDatum(PG_DETOAST_DATUM_COPY(*from));
+							else
+								*into = datumCopy(*from, false, typinfo->typlen[j]);
+						}
+						else
+							*into = *from;
+					}
+
+					if (!result->rangemaxnull[i])
+					{
+						Datum   *from = &src->rangemaxs[j][i];
+						Datum   *into = &result->rangemaxs[j][i];
+
+						if (!typinfo->typbyval[j])
+						{
+							if (typinfo->typlen[j] == -1)
+								*into = PointerGetDatum(PG_DETOAST_DATUM_COPY(*from));
+							else
+								*into = datumCopy(*from, false, typinfo->typlen[j]);
+						}
+						else
+							*into = *from;
+					}
+				}
+			}
+			break;
+	}
+
+	return result;
+}
+
+/*
+ * RelationGetPartitionInfo -- get the PartitionInfo containing info of
+ * existing partitions of a partitioned table
+ *
+ * We cache the partitions of relation as read from pg_partition. The cached
+ * info includes the array(s) of bounds, OIDs, etc. If relation is not
+ * partitioned, simply return NULL. Otherwise, the info is copied/created
+ * into/in the caller's memory context. (We don't want to return a pointer to
+ * the relcache copy, since it could disappear due to relcache invalidation.)
+ */
+PartitionInfo *
+RelationGetPartitionInfo(Relation relation)
+{
+	int		i,
+			j;
+	int		numparts = 0;
+	int		partnatts = relation->rd_pkey->partnatts;
+	char	strategy = relation->rd_pkey->partstrategy;
+	MemoryContext 	oldcxt;
+	PartitionInfo  *result = NULL;
+	PartitionBoundInfo **partitions = NULL;
+	PartitionKeyTypeInfo *typinfo;
+
+	/* Quick exit if there is nothing to do. */
+	if (relation->rd_pkeytuple == NULL)
+		return NULL;
+
+	/* Get the type info for partition keys. */
+	typinfo = get_key_type_info(relation);
+
+	/* Quick copy and exit if we already computed the result. */
+	if (relation->rd_partitions != NULL)
+		return CopyPartitionInfo(relation->rd_partitions,
+										relation->rd_pkey->partnatts,
+										typinfo);
+
+	/*
+	 * Either the first time building the list or the cached one was
+	 * recently invalidated, so build a new.
+	 */
+	partitions = GetPartitionBounds(relation, &numparts);
+
+	result = (PartitionInfo *) palloc0(sizeof(PartitionInfo));
+	result->numpartitions = numparts;
+	result->strategy = strategy;
+
+	/* Found some partitions. Now build a list in the caller's context. */
+	if (result->numpartitions)
+	{
+		result->oids = (Oid *) palloc0(numparts * sizeof(Oid));
+
+		switch (result->strategy)
+		{
+			case PARTITION_STRAT_LIST:
+				/* Simply copy list partitions. */
+				result->listnvalues = (int *) palloc0(numparts * sizeof(int));
+				result->listvalues = (Datum **) palloc0(numparts * sizeof(Datum *));
+
+				for (i = 0; i < numparts; i++)
+				{
+					result->oids[i] = partitions[i]->oid;
+					result->listnvalues[i] = partitions[i]->listnvalues;
+					result->listvalues[i] = (Datum *)
+								palloc0(partitions[i]->listnvalues * sizeof(Datum));
+					for (j = 0; j < result->listnvalues[i]; j++)
+					{
+						Datum   *from = &partitions[i]->listvalues[j];
+						Datum   *into = &result->listvalues[i][j];
+
+						if (!typinfo->typbyval[0])
+						{
+							if (typinfo->typlen[0] == -1)
+								*into = PointerGetDatum(PG_DETOAST_DATUM_COPY(*from));
+							else
+								*into = datumCopy(*from, false, typinfo->typlen[0]);
+						}
+						else
+							*into = *from;
+					}
+				}
+				break;
+
+			case PARTITION_STRAT_RANGE:
+				/* Sort on rangemax. */
+				qsort_arg(partitions, numparts,	sizeof(PartitionBoundInfo *),
+												range_partition_cmp_max,
+												relation->rd_partsupfuncs);
+				/* Copy */
+				result->rangeminnull = (bool *) palloc0(numparts * sizeof(bool));
+				result->rangemaxnull = (bool *) palloc0(numparts * sizeof(bool));
+				for (i = 0; i < partnatts; i++)
+				{
+					result->rangemins[i] = (Datum *) palloc0(numparts * sizeof(Datum));
+					result->rangemaxs[i] = (Datum *) palloc0(numparts * sizeof(Datum));
+				}
+
+				for (i = 0; i < numparts; i++)
+				{
+					result->oids[i] = partitions[i]->oid;
+					result->rangeminnull[i] = partitions[i]->rangeminnull;
+					result->rangemaxnull[i] = partitions[i]->rangemaxnull;
+
+					for (j = 0; j < partnatts; j++)
+					{
+						if (!result->rangeminnull[i])
+						{
+							Datum   *from = &partitions[i]->rangemins[j];
+							Datum   *into = &result->rangemins[j][i];
+
+							if (!typinfo->typbyval[j])
+							{
+								if (typinfo->typlen[j] == -1)
+									*into = PointerGetDatum(PG_DETOAST_DATUM_COPY(*from));
+								else
+									*into = datumCopy(*from, false, typinfo->typlen[j]);
+							}
+							else
+								*into = *from;
+						}
+
+						if (!result->rangemaxnull[i])
+						{
+							Datum   *from = &partitions[i]->rangemaxs[j];
+							Datum   *into = &result->rangemaxs[j][i];
+
+							if (!typinfo->typbyval[j])
+							{
+								if (typinfo->typlen[j] == -1)
+									*into = PointerGetDatum(PG_DETOAST_DATUM_COPY(*from));
+								else
+									*into = datumCopy(*from, false, typinfo->typlen[j]);
+							}
+							else
+								*into = *from;
+						}
+					}
+				}
+				break;
+		}
+		free_partitions(partitions, numparts);
+	}
+
+	/* Save a copy in the relcache entry of relation. */
+
+	/*
+	 * Create a memory context for caching the just created list of existing
+	 * partitions and copy the list. This cached context turns out to be a
+	 * convenience when dropping the cached list in RelationDestroyRelation.
+	 */
+    relation->rd_partlistcxt = AllocSetContextCreate(CacheMemoryContext,
+									RelationGetRelationName(relation),
+									ALLOCSET_DEFAULT_MINSIZE,
+									ALLOCSET_DEFAULT_INITSIZE,
+									ALLOCSET_DEFAULT_MAXSIZE);
+
+	oldcxt = MemoryContextSwitchTo(relation->rd_partlistcxt);
+	relation->rd_partitions = CopyPartitionInfo(result,
+												relation->rd_pkey->partnatts,
+												typinfo);
+	MemoryContextSwitchTo(oldcxt);
+
+	return result;
+}
+
+/*
+ * RelationBuildPartitionKey - initialize partition key related fields
+ *
+ * Sets the following fields of relation:
+ *
+ *	rd_pkey				The Form_pg_partitioned_rel entry
+ *	rd_pkeytuple		HeapTuple of the same
+ *	rd_pkeycxt			A long-lived context for the following items
+ *	rd_partopfamily		Operator family per key column
+ *	rd_partopcintype	Operator input type per key column
+ *	rd_partsupfuncs		Comparison functions per key column (support funcs)
+ *	rd_partexprs		Simply set to NIL. See RelationGetPartitionExpression
+ */
+static void
+RelationBuildPartitionKey(Relation relation)
+{
+	Relation		pg_partitioned_rel;
+	HeapTuple		tuple;
+	oidvector	   *partclass;
+	char			partstrategy;
+	int				partnatts;
+	int				i;
+	Datum			datum;
+	bool			isnull;
+	MemoryContext	pkeycxt;
+	MemoryContext	oldcxt;
+
+	tuple = SearchSysCache1(PARTITIONEDRELID, RelationGetRelid(relation));
+
+	/* pg_partitioned_rel entry just got dropped (shouldn't be the case)*/
+	if (!HeapTupleIsValid(tuple))
+	{
+		/* Safety! */
+		relation->rd_pkeytuple = NULL;
+		return;
+	}
+
+	/* Keep a copy of pg_partitioned_rel entry in the relation descriptor. */
+	oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+	relation->rd_pkeytuple = heap_copytuple(tuple);
+	relation->rd_pkey = (Form_pg_partitioned_rel) GETSTRUCT(relation->rd_pkeytuple);
+	MemoryContextSwitchTo(oldcxt);
+
+	/* pg_partitioned_rel tuple */
+	ReleaseSysCache(tuple);
+
+	partstrategy = relation->rd_pkey->partstrategy;
+	partnatts = relation->rd_pkey->partnatts;
+
+	/* Use partclass to figure out compare functions for each key column. */
+	pg_partitioned_rel = heap_open(PartitionedRelRelationId, AccessShareLock);
+
+	datum = fastgetattr(tuple,
+						Anum_pg_partitioned_rel_partclass,
+						RelationGetDescr(pg_partitioned_rel),
+						&isnull);
+
+	Assert(!isnull);
+	partclass = (oidvector *) DatumGetPointer(datum);
+
+	heap_close(pg_partitioned_rel, AccessShareLock);
+
+    /*
+	 * Make the private context to cache the items we compute below. The reason
+	 * we need a context, and not just a couple of pallocs, is so that we won't
+	 * leak any subsidiary info attached to fmgr lookup records.
+	 *
+	 * Context parameters are set on the assumption that it'll probably not
+	 * contain much data.
+	 */
+    pkeycxt = AllocSetContextCreate(CacheMemoryContext,
+									RelationGetRelationName(relation),
+									ALLOCSET_SMALL_MINSIZE,
+									ALLOCSET_SMALL_INITSIZE,
+									ALLOCSET_SMALL_MAXSIZE);
+	relation->rd_pkeycxt = pkeycxt;
+
+	relation->rd_partopfamily = (Oid *) MemoryContextAllocZero(pkeycxt,
+												partnatts * sizeof(Oid));
+	relation->rd_partopcintype = (Oid *) MemoryContextAllocZero(pkeycxt,
+												partnatts * sizeof(Oid));
+	relation->rd_partsupfuncs = (FmgrInfo *) MemoryContextAllocZero(pkeycxt,
+												partnatts * sizeof(FmgrInfo));
+
+	/* Determine the opfamily and opcintype. */
+	for (i = 0; i < partnatts; i++)
+	{
+		HeapTuple			tuple;
+		Form_pg_opclass 	form;
+		Oid					funcid;
+
+		tuple = SearchSysCache(CLAOID,
+						ObjectIdGetDatum(partclass->values[i]),
+						0, 0, 0);
+		if (!HeapTupleIsValid(tuple))
+				elog(ERROR, "cache lookup failed for opclass %u",
+						partclass->values[i]);
+
+		form = (Form_pg_opclass) GETSTRUCT(tuple);
+
+		relation->rd_partopfamily[i] = form->opcfamily;
+		relation->rd_partopcintype[i] = form->opcintype;
+
+		/*
+		 * cache a btree support function for inter-partition comparisons
+		 * as well as comparing a new tuple against partition bounds.
+		 */
+		funcid = get_opfamily_proc(form->opcfamily,
+								form->opcintype, form->opcintype,
+								BTORDER_PROC);
+
+		fmgr_info(funcid, &relation->rd_partsupfuncs[i]);
+
+		ReleaseSysCache(tuple);
+	}
+
+	/* RelationGetPartitionExpressions() caches its result here. */
+	relation->rd_partexprs = NIL;
+}
 
 /*
  * Routines to support ereport() reports of relation-related errors
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index efce7b9..3aa2d43 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -47,6 +47,8 @@
 #include "catalog/pg_opclass.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
+#include "catalog/pg_partition.h"
+#include "catalog/pg_partitioned_rel.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_range.h"
 #include "catalog/pg_rewrite.h"
@@ -567,6 +569,28 @@ static const struct cachedesc cacheinfo[] = {
 		},
 		8
 	},
+	{PartitionRelationId,		/* PARTITIONID */
+		PartitionIdIndexId,
+		1,
+		{
+			Anum_pg_partition_partitionid,
+			0,
+			0,
+			0
+		},
+		64
+	},
+	{PartitionedRelRelationId,		/* PARTITIONEDRELID */
+		PartitionedRelrelidIndexId,
+		1,
+		{
+			Anum_pg_partitioned_rel_partrelid,
+			0,
+			0,
+			0
+		},
+		64
+	},
 	{ProcedureRelationId,		/* PROCNAMEARGSNSP */
 		ProcedureNameArgsNspIndexId,
 		3,
diff --git a/src/backend/utils/misc/Makefile b/src/backend/utils/misc/Makefile
index 7889101..473d071 100644
--- a/src/backend/utils/misc/Makefile
+++ b/src/backend/utils/misc/Makefile
@@ -14,7 +14,7 @@ include $(top_builddir)/src/Makefile.global
 
 override CPPFLAGS := -I. -I$(srcdir) $(CPPFLAGS)
 
-OBJS = guc.o help_config.o pg_rusage.o ps_status.o rls.o \
+OBJS = guc.o help_config.o partition.o pg_rusage.o ps_status.o rls.o \
        sampling.o superuser.o timeout.o tzparser.o
 
 # This location might depend on the installation directories. Therefore
diff --git a/src/backend/utils/misc/partition.c b/src/backend/utils/misc/partition.c
new file mode 100644
index 0000000..301d6af
--- /dev/null
+++ b/src/backend/utils/misc/partition.c
@@ -0,0 +1,465 @@
+/*-------------------------------------------------------------------------
+ *
+ * partition.c
+ *        Partitioning related utility functions.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *        src/backend/utils/misc/partition.c
+ *
+ *-------------------------------------------------------------------------
+*/
+#include "postgres.h"
+
+#include "catalog/heap.h"
+#include "catalog/pg_collation.h"
+#include "executor/executor.h"
+#include "miscadmin.h"
+#include "nodes/nodeFuncs.h"
+#include "utils/partition.h"
+#include "utils/lsyscache.h"
+#include "utils/relcache.h"
+
+static Oid find_partition_for_tuple_recurse(Relation rel, TupleTableSlot *slot,
+								EState *estate);
+static Oid find_list_partition_for_tuple(Relation rel, PartitionInfo *pinfo,
+								Datum value);
+static Oid find_range_partition_for_tuple(Relation rel, PartitionInfo *pinfo,
+								Datum *values);
+
+/*
+ * get_partition_key_type_info
+ *
+ * Returns a PartitionKeyTypeInfo object that includes type info
+ * for individual partition key attributes.
+ *
+ * Separated out to avoid repeating the code.
+ */
+PartitionKeyTypeInfo *get_key_type_info(Relation rel)
+{
+	int			i;
+	int			partnatts;
+	AttrNumber *partattrs;
+	AttrNumber	attno;
+	List	   *partexprs;
+	ListCell   *partexprs_item;
+	PartitionKeyTypeInfo *result;
+
+	result = (PartitionKeyTypeInfo *) palloc0(sizeof(PartitionKeyTypeInfo));
+
+	partnatts = rel->rd_pkey->partnatts;
+	partattrs = rel->rd_pkey->partkey.values;
+	partexprs = RelationGetPartitionExpressions(rel);
+
+	partexprs_item = list_head(partexprs);
+	for (i = 0; i < partnatts; i++)
+	{
+		attno = partattrs[i];
+
+		if (attno != InvalidAttrNumber)
+		{
+			result->typid[i] = rel->rd_att->attrs[attno - 1]->atttypid;
+			result->typmod[i] = rel->rd_att->attrs[attno - 1]->atttypmod;
+		}
+		else
+		{
+			result->typid[i] = exprType(lfirst(partexprs_item));
+			result->typmod[i] = exprTypmod(lfirst(partexprs_item));
+			partexprs_item = lnext(partexprs_item);
+		}
+		get_typlenbyvalalign(result->typid[i],
+								&result->typlen[i],
+								&result->typbyval[i],
+								&result->typalign[i]);
+	}
+
+	return result;
+}
+
+/*
+ * range_partition_cmp_max - compare two range partitions using
+ *							their max bounds
+ *
+ * 'a' and 'b' point to the range partitions being compared.
+ *
+ * 'arg' is an array of pointers to FmgrInfo structs, one for each
+ * comparison function corresponding to partition key columns.
+ */
+int32
+range_partition_cmp_max(const void *a, const void *b, void *arg)
+{
+	int			i;
+	int32		result;
+	FmgrInfo   *cmpfn = (FmgrInfo *) arg;
+	int			natts = (*(const PartitionBoundInfo **) a)->partnatts;
+	const PartitionBoundInfo *arg1 = *(const PartitionBoundInfo **) a;
+	const PartitionBoundInfo *arg2 = *(const PartitionBoundInfo **) b;
+
+	/* rangemaxnull implies rangemax is +INFINITY */
+
+	/* arg1 > any, given it is an existing partition */
+	if (arg1->rangemaxnull && arg1->oid != InvalidOid)
+		return +1;
+
+	/* arg2 > any, given it is an existing partition */
+	if (arg2->rangemaxnull && arg2->oid != InvalidOid)
+		return -1;
+
+	for (i = 0; i < natts; i++)
+	{
+		result = DatumGetInt32(FunctionCall2Coll(&cmpfn[i],
+						DEFAULT_COLLATION_OID,
+						arg1->rangemaxs[i], arg2->rangemaxs[i]));
+
+		/* consider multicolumn range partitions */
+		if (!result)
+			continue;
+		else
+			return result;
+	}
+
+	/*
+	 * A new partition being added is assumed to order before an existing
+	 * partition with the equal bound for the assumptions made in
+	 * CheckPartitionAgainstExisting() about partition ordering.
+	 *
+	 * XXX - clarify the assumption?
+	 */
+	if (arg1->oid == InvalidOid)
+		return -1;
+
+	if (arg2->oid == InvalidOid)
+		return +1;
+
+	return 0;
+}
+
+/*
+ * range_partition_cmp_min - compare two range partitions using
+ *							their min bounds
+ *
+ * 'a' and 'b' point to the range partitions being compared.
+ *
+ * 'arg' is an array of pointers to FmgrInfo structs, one for each
+ * comparison function corresponding to partition key columns.
+ */
+int32
+range_partition_cmp_min(const void *a, const void *b, void *arg)
+{
+	int			i;
+	int32		result;
+	FmgrInfo   *cmpfn = (FmgrInfo *) arg;
+	int			natts = (*(const PartitionBoundInfo **) a)->partnatts;
+	const PartitionBoundInfo *arg1 = *(const PartitionBoundInfo **) a;
+	const PartitionBoundInfo *arg2 = *(const PartitionBoundInfo **) b;
+
+	/* rangeminnull implies rangemin is -INFINITY */
+
+	/* arg1 < any, given it is an existing partition */
+	if (arg1->rangeminnull && arg1->oid != InvalidOid)
+		return -1;
+
+	/* arg2 < any, given it is an existing partition*/
+	if (arg2->rangeminnull && arg2->oid != InvalidOid)
+		return +1;
+
+	for (i = 0; i < natts; i++)
+	{
+		result = DatumGetInt32(FunctionCall2Coll(&cmpfn[i],
+						DEFAULT_COLLATION_OID,
+						arg1->rangemins[i], arg2->rangemins[i]));
+
+		/* Consider multicolumn range partitions.*/
+		if (!result)
+			continue;
+		else
+			return result;
+	}
+
+	/*
+	 * A new partition being added is assumed to order after an existing
+	 * partition with the equal bound for the assumptions made in
+	 * CheckPartitionAgainstExisting() about partition ordering.
+	 *
+	 * XXX - clarify the assumption?
+	 */
+	if (arg1->oid == InvalidOid)
+		return +1;
+
+	if (arg2->oid == InvalidOid)
+		return -1;
+
+	return 0;
+}
+
+/*
+ * range_partition_cmp_bounds - compare bounds (min or max) of two
+ *						partitions that are both known not NULL
+ *
+ * 'arg1' and 'arg2' are datum arrays, each 'natts' datums long.
+ * They could be a min or max bound, each belonging to a differnt
+ * partition.
+ *
+ * 'cmpfn' is an array of pointers to FmgrInfo structs, one for each
+ * comparison function corresponding to partition key columns.
+ *
+ * Note: This functions is directly called using rangemins/rangemaxs
+ * that are known not NULL (applies to arg1 and/or arg2).
+ */
+int32
+range_partition_cmp_bounds(const Datum *arg1, const Datum *arg2,
+							int natts,
+							FmgrInfo *cmpfn)
+{
+	int 	i;
+	int32	result;
+
+	for (i = 0; i < natts; i++)
+	{
+		result = DatumGetInt32(FunctionCall2Coll(&cmpfn[i],
+									DEFAULT_COLLATION_OID,
+									arg1[i], arg2[i]));
+
+		/* consider multicolumn range partitions */
+		if (!result)
+			continue;
+		else
+			return result;
+	}
+
+	return 0;
+}
+
+/*
+ * list_partitions_overlap - does list partition a overlap b?
+ *
+ * 'arg' is an array of pointers to FmgrInfo structs, one for each
+ * comparison function corresponding to partition key columns.
+ */
+bool
+list_partition_overlaps(PartitionBoundInfo *a, PartitionBoundInfo *b,
+										FmgrInfo *cmpfn)
+{
+	int			i,
+				j;
+	int32		result;
+
+	/* For each value in a's list, check if it's equal to any of b's */
+	for (i = 0; i < a->listnvalues; i++)
+	{
+		Datum	aDatum = a->listvalues[i];
+
+		for (j = 0; j < b->listnvalues; j++)
+		{
+			/* Note: There is only ever one cmpfn for list partitioning. */
+			result = DatumGetInt32(FunctionCall2Coll(&cmpfn[0],
+							DEFAULT_COLLATION_OID,
+							aDatum, b->listvalues[j]));
+			if (!result)
+				return true;
+		}
+	}
+
+	return false;
+}
+
+/*
+ * get_partition_for_tuple - entry point for recursively determining
+ * the target partition for tuple.
+ */
+Oid
+find_partition_for_tuple(Relation rel,
+							PartitionKeyInfo *pkinfo,
+							PartitionInfo *pinfo,
+							TupleTableSlot *slot,
+							EState *estate)
+{
+	int	i;
+	Oid					targetOid,
+						relid = RelationGetRelid(rel);
+	Datum				values[PARTITION_MAX_KEYS];
+	bool				isnull[PARTITION_MAX_KEYS];
+	TupleTableSlot	   *ecxt_scantuple;
+
+	/* Extract partition key column values from tuple */
+	Assert(GetPerTupleExprContext(estate)->ecxt_scantuple == slot);
+	FormPartitionKeyDatum(pkinfo,
+							slot,
+							estate,
+							values,
+							isnull);
+
+	/* For time being, disallow nulls in partition key cols. */
+	for (i = 0; i < pkinfo->pi_NumKeyAttrs; i++)
+		if (isnull[i])
+			ereport(ERROR,
+					(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+					 errmsg("null value for partition key column \"%s\" not allowed",
+							get_attname(relid, pkinfo->pi_KeyAttrNumbers[i]))));
+
+	/* Find out actual partition. */
+	switch (pinfo->strategy)
+	{
+		case PARTITION_STRAT_LIST:
+			targetOid = find_list_partition_for_tuple(rel, pinfo, values[0]);
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			targetOid = find_range_partition_for_tuple(rel, pinfo, values);
+			break;
+	}
+
+	/*
+	 * Recurse to find actual partition if this partition itself is
+	 * partitioned.
+	 */
+	if (targetOid && is_partitioned(targetOid))
+	{
+		Relation	relation;
+		relation = heap_open(targetOid, RowExclusiveLock);
+		targetOid = find_partition_for_tuple_recurse(relation, slot, estate);
+		/* XXX - should be closed now? */
+		heap_close(relation, RowExclusiveLock);
+	}
+
+	return targetOid;
+}
+
+/*
+ * find_partition_for_tuple_recurse - performs recursion for when a partition
+ * itself is found to be partitioned.
+ */
+
+static Oid
+find_partition_for_tuple_recurse(Relation rel,
+								TupleTableSlot *slot,
+								EState *estate)
+{
+	PartitionKeyInfo   *pkinfo;
+	PartitionInfo	   *pinfo;
+
+	check_stack_depth();
+
+	pkinfo = BuildPartitionKeyInfo(rel);
+	pinfo = RelationGetPartitionInfo(rel);
+
+	return find_partition_for_tuple(rel, pkinfo, pinfo, slot, estate);
+}
+
+/*
+ * find_list_partition_for_tuple - find the list partition for value
+ */
+static Oid
+find_list_partition_for_tuple(Relation rel, PartitionInfo *pinfo,
+								Datum value)
+{
+	int			i,
+				j;
+	int32		result;
+	FmgrInfo	cmpfn = rel->rd_partsupfuncs[0];
+
+	if (pinfo->numpartitions == 0)
+		return InvalidOid;
+
+	for (i = 0; i < pinfo->numpartitions; i++)
+	{
+		for (j = 0; j < pinfo->listnvalues[i]; j++)
+		{
+			result = DatumGetInt32(FunctionCall2Coll(&cmpfn,
+						DEFAULT_COLLATION_OID,
+						pinfo->listvalues[i][j], value));
+
+			if (!result)
+				return pinfo->oids[i];
+		}
+	}
+
+	return InvalidOid;
+}
+
+/*
+ * find_range_partition_for_tuple - find the range partition for values
+ */
+static Oid
+find_range_partition_for_tuple(Relation rel, PartitionInfo *pinfo,
+								Datum *values)
+{
+	int		low, high, idx,
+			i,
+			partnatts = rel->rd_pkey->partnatts;
+	FmgrInfo   *cmpfn = rel->rd_partsupfuncs;
+	int32		cmpval;
+	Datum	   *rangebound;
+
+	if (pinfo->numpartitions == 0)
+		return InvalidOid;
+
+	rangebound = (Datum *) palloc0(partnatts * sizeof(Datum));
+
+	/* Good ol' bsearch */
+	low = 0;
+	high = pinfo->numpartitions - 1;
+	while (low <= high)
+	{
+		idx = (low + high) / 2;
+
+		/* Note: Only the last partition is allowed to have null rangemax. */
+		if (pinfo->rangemaxnull[idx])
+		{
+			for (i = 0; i < partnatts; i++)
+				rangebound[i] = pinfo->rangemins[i][idx];
+
+			cmpval = range_partition_cmp_bounds(values, rangebound,
+													partnatts, cmpfn);
+			if (cmpval >= 0)
+				return pinfo->oids[idx];
+			else
+				return InvalidOid;
+		}
+		else
+		{
+			for (i = 0; i < partnatts; i++)
+				rangebound[i] = pinfo->rangemaxs[i][idx];
+
+			cmpval = range_partition_cmp_bounds(values, rangebound,
+													partnatts, cmpfn);
+			if (!cmpval)
+			{
+				for (i = 0; i < partnatts; i++)
+					rangebound[i] = pinfo->rangemins[i][idx+1];
+
+				cmpval = range_partition_cmp_bounds(values, rangebound,
+														partnatts, cmpfn);
+				if (!cmpval)
+					return pinfo->oids[idx+1];
+				else	/* There is a gap between this and the next. */
+					return InvalidOid;
+			}
+			else if (cmpval < 0)
+			{
+				/* Note: only the first partition is allowed to have null rangemin. */
+				if (pinfo->rangeminnull[idx])
+					return pinfo->oids[idx];
+				else
+				{
+					for (i = 0; i < partnatts; i++)
+						rangebound[i] = pinfo->rangemins[i][idx];
+
+					cmpval = range_partition_cmp_bounds(values, rangebound,
+															partnatts, cmpfn);
+					if (cmpval >= 0)
+						return pinfo->oids[idx];
+					else
+						high = idx - 1;
+				}
+			}
+			else
+				low = idx + 1;
+		}
+	}
+
+	return InvalidOid;
+}
diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h
index e6ac394..fb1af70 100644
--- a/src/include/catalog/heap.h
+++ b/src/include/catalog/heap.h
@@ -17,6 +17,7 @@
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "parser/parse_node.h"
+#include "nodes/execnodes.h"
 
 
 typedef struct RawColumnDefault
@@ -133,5 +134,20 @@ extern void CheckAttributeType(const char *attname,
 				   Oid atttypid, Oid attcollation,
 				   List *containing_rowtypes,
 				   bool allow_system_table_mods);
+extern void StorePartitionKey(Relation rel, PartitionKeyInfo *pkinfo,
+					Oid *partClassOids, char strategy);
+extern void RemovePartitionKeyByRelId(Oid relid);
+extern PartitionKeyInfo *BuildPartitionKeyInfo(Relation rel);
+extern void RemovePartitionBoundByRelId(Oid relid);
+extern void StorePartitionBounds(Relation childrel,
+					Relation parentrel,
+					int listnvalues,
+					Datum *datum);
+extern void SetRelationIsPartition(Relation rel, bool ispartition);
+extern void	FormPartitionKeyDatum(PartitionKeyInfo *pkinfo,
+					TupleTableSlot *slot,
+					EState *estate,
+					Datum *values,
+					bool *isnull);
 
 #endif   /* HEAP_H */
diff --git a/src/include/catalog/indexing.h b/src/include/catalog/indexing.h
index c38958d..1f73595 100644
--- a/src/include/catalog/indexing.h
+++ b/src/include/catalog/indexing.h
@@ -316,6 +316,14 @@ DECLARE_UNIQUE_INDEX(pg_replication_origin_roiident_index, 6001, on pg_replicati
 DECLARE_UNIQUE_INDEX(pg_replication_origin_roname_index, 6002, on pg_replication_origin using btree(roname text_pattern_ops));
 #define ReplicationOriginNameIndex 6002
 
+DECLARE_UNIQUE_INDEX(pg_partitioned_rel_partrelid_index, 3315, on pg_partitioned_rel using btree(partrelid oid_ops));
+#define PartitionedRelrelidIndexId				3315
+
+DECLARE_UNIQUE_INDEX(pg_partition_partitionid_index, 3316, on pg_partition using btree(partitionid oid_ops));
+#define PartitionIdIndexId			3316
+DECLARE_INDEX(pg_partition_parent_index, 3317, on pg_partition using btree(partparent oid_ops));
+#define PartitionParentIndexId		3317
+
 /* last step of initialization script: build the indexes declared above */
 BUILD_INDICES
 
diff --git a/src/include/catalog/pg_class.h b/src/include/catalog/pg_class.h
index e526cd9..a9a9e0d 100644
--- a/src/include/catalog/pg_class.h
+++ b/src/include/catalog/pg_class.h
@@ -65,7 +65,8 @@ CATALOG(pg_class,1259) BKI_BOOTSTRAP BKI_ROWTYPE_OID(83) BKI_SCHEMA_MACRO
 	bool		relhasrules;	/* has (or has had) any rules */
 	bool		relhastriggers; /* has (or has had) any TRIGGERs */
 	bool		relhassubclass; /* has (or has had) derived classes */
-	bool		relrowsecurity; /* row security is enabled or not */
+	bool		relispartition; /* is a RELKIND_RELATION relation a partition */
+	bool		relrowsecurity;	/* row security is enabled or not */
 	bool		relispopulated; /* matview currently holds query results */
 	char		relreplident;	/* see REPLICA_IDENTITY_xxx constants  */
 	TransactionId relfrozenxid; /* all Xids < this are frozen in this rel */
@@ -95,7 +96,7 @@ typedef FormData_pg_class *Form_pg_class;
  * ----------------
  */
 
-#define Natts_pg_class					30
+#define Natts_pg_class					31
 #define Anum_pg_class_relname			1
 #define Anum_pg_class_relnamespace		2
 #define Anum_pg_class_reltype			3
@@ -119,13 +120,14 @@ typedef FormData_pg_class *Form_pg_class;
 #define Anum_pg_class_relhasrules		21
 #define Anum_pg_class_relhastriggers	22
 #define Anum_pg_class_relhassubclass	23
-#define Anum_pg_class_relrowsecurity	24
-#define Anum_pg_class_relispopulated	25
-#define Anum_pg_class_relreplident		26
-#define Anum_pg_class_relfrozenxid		27
-#define Anum_pg_class_relminmxid		28
-#define Anum_pg_class_relacl			29
-#define Anum_pg_class_reloptions		30
+#define Anum_pg_class_relispartition	24
+#define Anum_pg_class_relrowsecurity	25
+#define Anum_pg_class_relispopulated	26
+#define Anum_pg_class_relreplident		27
+#define Anum_pg_class_relfrozenxid		28
+#define Anum_pg_class_relminmxid		29
+#define Anum_pg_class_relacl			30
+#define Anum_pg_class_reloptions		31
 
 /* ----------------
  *		initial contents of pg_class
@@ -140,13 +142,13 @@ typedef FormData_pg_class *Form_pg_class;
  * Note: "3" in the relfrozenxid column stands for FirstNormalTransactionId;
  * similarly, "1" in relminmxid stands for FirstMultiXactId
  */
-DATA(insert OID = 1247 (  pg_type		PGNSP 71 0 PGUID 0 0 0 0 0 0 0 f f p r 30 0 t f f f f f t n 3 1 _null_ _null_ ));
+DATA(insert OID = 1247 (  pg_type		PGNSP 71 0 PGUID 0 0 0 0 0 0 0 f f p r 30 0 t f f f f f f t n 3 1 _null_ _null_ ));
 DESCR("");
-DATA(insert OID = 1249 (  pg_attribute	PGNSP 75 0 PGUID 0 0 0 0 0 0 0 f f p r 21 0 f f f f f f t n 3 1 _null_ _null_ ));
+DATA(insert OID = 1249 (  pg_attribute	PGNSP 75 0 PGUID 0 0 0 0 0 0 0 f f p r 21 0 f f f f f f f t n 3 1 _null_ _null_ ));
 DESCR("");
-DATA(insert OID = 1255 (  pg_proc		PGNSP 81 0 PGUID 0 0 0 0 0 0 0 f f p r 28 0 t f f f f f t n 3 1 _null_ _null_ ));
+DATA(insert OID = 1255 (  pg_proc		PGNSP 81 0 PGUID 0 0 0 0 0 0 0 f f p r 28 0 t f f f f f f t n 3 1 _null_ _null_ ));
 DESCR("");
-DATA(insert OID = 1259 (  pg_class		PGNSP 83 0 PGUID 0 0 0 0 0 0 0 f f p r 30 0 t f f f f f t n 3 1 _null_ _null_ ));
+DATA(insert OID = 1259 (  pg_class		PGNSP 83 0 PGUID 0 0 0 0 0 0 0 f f p r 31 0 t f f f f f f t n 3 1 _null_ _null_ ));
 DESCR("");
 
 
@@ -158,6 +160,7 @@ DESCR("");
 #define		  RELKIND_COMPOSITE_TYPE  'c'		/* composite type */
 #define		  RELKIND_FOREIGN_TABLE   'f'		/* foreign table */
 #define		  RELKIND_MATVIEW		  'm'		/* materialized view */
+#define		  RELKIND_PARTITIONED_REL 'P'		/* partitioned table */
 
 #define		  RELPERSISTENCE_PERMANENT	'p'		/* regular table */
 #define		  RELPERSISTENCE_UNLOGGED	'u'		/* unlogged permanent table */
diff --git a/src/include/catalog/pg_partition.h b/src/include/catalog/pg_partition.h
new file mode 100644
index 0000000..2c5b736
--- /dev/null
+++ b/src/include/catalog/pg_partition.h
@@ -0,0 +1,61 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_partition.h
+ *	  definition of the system "partition" relation (pg_partition)
+ *	  along with the relation's initial contents.
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ *
+ * $PostgreSQL: pgsql/src/include/catalog/pg_partition.h $
+ *
+ * NOTES
+ *	  the genbki.sh script reads this file and generates .bki
+ *	  information from the DATA() statements.
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_PARTITION_H
+#define PG_PARTITION_H
+
+#include "catalog/genbki.h"
+
+/* ----------------
+ *		pg_partitioned_rel definition.  cpp turns this into
+ *		typedef struct FormData_pg_partitioned_rel
+ * ----------------
+ */
+#define PartitionRelationId 3309
+
+CATALOG(pg_partition,3309) BKI_WITHOUT_OIDS
+{
+	Oid			partitionid;		/* partition oid */
+	Oid			partparent;		/* parent oid */
+
+#ifdef CATALOG_VARLEN				/* variable-length fields start here */
+	anyarray	partlistvalues;		/* list of allowed values of the only
+									 * partition column */
+	anyarray	partrangebounds;	/* list of bounds of ranges of
+									 * allowed values per partition key
+									 * column */
+#endif
+} FormData_pg_partition;
+
+/* ----------------
+ *      Form_pg_partition corresponds to a pointer to a tuple with
+ *      the format of pg_partition relation.
+ * ----------------
+ */
+typedef FormData_pg_partition *Form_pg_partition;
+
+/* ----------------
+ *      compiler constants for pg_partition
+ * ----------------
+ */
+#define Natts_pg_partition				4
+#define Anum_pg_partition_partitionid		1
+#define Anum_pg_partition_partparent		2
+#define Anum_pg_partition_partlistvalues	3
+#define Anum_pg_partition_partrangebounds	4
+
+#endif   /* PG_PARTITION_H */
diff --git a/src/include/catalog/pg_partition_fn.h b/src/include/catalog/pg_partition_fn.h
new file mode 100644
index 0000000..b038aa2
--- /dev/null
+++ b/src/include/catalog/pg_partition_fn.h
@@ -0,0 +1,51 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_partition_fn.h
+ *	  prototypes for functions in catalog/pg_partition.c
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/catalog/pg_partition_fn.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_PARTITION_FN_H
+#define PG_PARTITION_FN_H
+
+#include "nodes/pg_list.h"
+#include "nodes/execnodes.h"
+#include "storage/lock.h"
+#include "utils/rel.h"
+
+/* Bound info of a single partition */
+typedef struct PartitionBoundInfo
+{
+	Oid		oid;			/* This partition's OID */
+	int		partnatts;		/* qsort_arg needs this to iterate over key columns
+							   when comparing two PartitionBoundInfo objects */
+	int		listnvalues;	/* How many values does listvalues below hold */
+	bool	rangeminnull;	/* Whether this range partition does not have a
+							   min bound */
+	bool	rangemaxnull;	/* Whether this range partition does not have a
+							   max bound */
+	Datum  *rangemins;		/* A range partition's min bound - one datum per
+							   key column */
+	Datum  *rangemaxs;		/* A range partition's max bound - one datum per
+							   key column */
+	Datum  *listvalues;		/* A list partition's values - the distinct values
+							   that the only key column can take */
+} PartitionBoundInfo;
+
+extern PartitionBoundInfo **GetPartitionBounds(Relation rel, int *numparts);
+extern void free_partitions(PartitionBoundInfo **p, int count);
+extern Oid get_partition_parent(Oid relid, bool recurse);
+extern bool IsPartition(Relation rel);
+extern bool is_partition(Oid relid);
+extern bool IsPartitionedRel(Relation rel);
+extern bool is_partitioned(Oid relid);
+extern List *find_partitions(Oid relid, LOCKMODE lockmode);
+extern List *find_all_partitions(Oid relid, LOCKMODE lockmode);
+
+#endif   /* PG_PARTITION_FN_H */
diff --git a/src/include/catalog/pg_partitioned_rel.h b/src/include/catalog/pg_partitioned_rel.h
new file mode 100644
index 0000000..dc55bea
--- /dev/null
+++ b/src/include/catalog/pg_partitioned_rel.h
@@ -0,0 +1,66 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_partitioned_rel.h
+ *	  definition of the system "partitioned" relation (pg_partitioned_rel)
+ *	  along with the relation's initial contents.
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ *
+ * $PostgreSQL: pgsql/src/include/catalog/pg_partitioned_rel.h $
+ *
+ * NOTES
+ *	  the genbki.sh script reads this file and generates .bki
+ *	  information from the DATA() statements.
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PG_PARTITIONED_REL_H
+#define PG_PARTITIONED_REL_H
+
+#include "catalog/genbki.h"
+
+/* ----------------
+ *		pg_partitioned_rel definition.  cpp turns this into
+ *		typedef struct FormData_pg_partitioned_rel
+ * ----------------
+ */
+#define PartitionedRelRelationId 3308
+
+CATALOG(pg_partitioned_rel,3308) BKI_WITHOUT_OIDS
+{
+	Oid				partrelid;		/* partitioned table oid */
+	char			partstrategy;	/* partitioning strategy */
+	int16			partnatts;		/* number of partition columns */
+
+	/* variable-length fields start here, but we allow direct access to indkey */
+	int2vector		partkey;		/* column numbers of partition columns */
+
+#ifdef CATALOG_VARLEN
+	oidvector		partclass;		/* operator class to compare keys */
+	pg_node_tree	partexprs;		/* expression trees for partition key members
+									 * that are not simple column references; one
+									 * for each zero entry in partkey[] */
+#endif
+} FormData_pg_partitioned_rel;
+
+/* ----------------
+ *      Form_pg_partitioned_rel corresponds to a pointer to a tuple with
+ *      the format of pg_partitioned_rel relation.
+ * ----------------
+ */
+typedef FormData_pg_partitioned_rel *Form_pg_partitioned_rel;
+
+/* ----------------
+ *      compiler constants for pg_partitioned_rel
+ * ----------------
+ */
+#define Natts_pg_partitioned_rel				6
+#define Anum_pg_partitioned_rel_partrelid		1
+#define Anum_pg_partitioned_rel_partstrategy	2
+#define Anum_pg_partitioned_rel_partnatts		3
+#define Anum_pg_partitioned_rel_partkey			4
+#define Anum_pg_partitioned_rel_partclass		5
+#define Anum_pg_partitioned_rel_partexprs		6
+
+#endif   /* PG_PARTITIONED_REL_H */
diff --git a/src/include/commands/defrem.h b/src/include/commands/defrem.h
index adae296..b998945 100644
--- a/src/include/commands/defrem.h
+++ b/src/include/commands/defrem.h
@@ -42,6 +42,8 @@ extern bool CheckIndexCompatible(Oid oldId,
 					 List *attributeList,
 					 List *exclusionOpNames);
 extern Oid	GetDefaultOpClass(Oid type_id, Oid am_id);
+extern Oid	GetIndexOpClass(List *opclass, Oid attrType, const char *accessMethodName,
+					Oid accessMethodId);
 
 /* commands/functioncmds.c */
 extern ObjectAddress CreateFunction(CreateFunctionStmt *stmt, const char *queryString);
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 193a654..c1d4f48 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -375,6 +375,9 @@ extern void check_exclusion_constraint(Relation heap, Relation index,
 						   ItemPointer tupleid,
 						   Datum *values, bool *isnull,
 						   EState *estate, bool newIndex);
+extern ResultRelInfo *ExecFindPartition(ResultRelInfo *resultRelInfo,
+							TupleTableSlot *slot,
+							EState *estate);
 
 
 #endif   /* EXECUTOR_H  */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 5796de8..a281e21 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -20,6 +20,7 @@
 #include "lib/pairingheap.h"
 #include "nodes/params.h"
 #include "nodes/plannodes.h"
+#include "utils/rel.h"
 #include "utils/reltrigger.h"
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
@@ -76,6 +77,28 @@ typedef struct IndexInfo
 	bool		ii_BrokenHotChain;
 } IndexInfo;
 
+/*
+ * PartitionKeyInfo
+ *
+ *		This struct holds the information needed to extract partition
+ *		column values from a heap tuple. Also (ab-)used by tablecmds.c to
+ *		to pass around the key info when defining a partitioned relation.
+ *
+ *		NumKeyAttrs			number of columns in partition key
+ *		KeyAttrNumbers		underlying-rel attribute numbers used as keys
+ *							(zeroes indicate expressions)
+ *		Expressions			expr trees for expression entries, or NIL if none
+ *		ExpressionsState	exec state for expressions, or NIL if none
+ */
+typedef struct PartitionKeyInfo
+{
+	NodeTag		type;
+	int			pi_NumKeyAttrs;
+	AttrNumber	pi_KeyAttrNumbers[PARTITION_MAX_KEYS];
+	List	   *pi_Expressions;			/* list of Expr */
+	List	   *pi_ExpressionsState;	/* list of ExprState */
+} PartitionKeyInfo;
+
 /* ----------------
  *	  ExprContext_CB
  *
@@ -341,6 +364,8 @@ typedef struct ResultRelInfo
 	ProjectionInfo *ri_projectReturning;
 	ProjectionInfo *ri_onConflictSetProj;
 	List	   *ri_onConflictSetWhere;
+	PartitionKeyInfo *ri_PartitionKeyInfo;
+	PartitionInfo  *ri_Partitions;
 } ResultRelInfo;
 
 /* ----------------
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 748e434..ee3c12d 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -37,6 +37,7 @@ typedef enum NodeTag
 	T_ResultRelInfo,
 	T_EState,
 	T_TupleTableSlot,
+	T_PartitionKeyInfo,
 
 	/*
 	 * TAGS FOR PLAN NODES (plannodes.h)
@@ -427,6 +428,10 @@ typedef enum NodeTag
 	T_OnConflictClause,
 	T_CommonTableExpr,
 	T_RoleSpec,
+	T_PartitionBy,
+	T_PartitionElem,
+	T_PartitionValues,
+	T_PartitionDef,
 
 	/*
 	 * TAGS FOR REPLICATION GRAMMAR PARSE NODES (replnodes.h)
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 151c93a..954ef4e 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -689,6 +689,76 @@ typedef struct XmlSerialize
 	int			location;		/* token location, or -1 if unknown */
 } XmlSerialize;
 
+/* Partitioning related definitions */
+
+/*
+ * PartitionElem - a partition key column
+ *
+ *		CREATE TABLE ... PARTITION BY ... ON (<here>)
+ *
+ *	'name'		Name of the table column included in the key
+ *	'expr'		Expression node tree of expressional key column
+ *	'opclass'	Operator class name associated with the column
+ */
+typedef struct PartitionElem
+{
+	NodeTag		type;
+	char	   *name;		/* name of column to partition on, or NULL */
+	Node	   *expr;		/* expression to partition on, or NULL */
+	List	   *opclass;	/* name of desired opclass; NIL = default */
+	int			location;	/* token location, or -1 if unknown */
+} PartitionElem;
+
+/*
+ * PartitionBy - partition key definition including the strategy
+ *
+ *	'strategy'		partition strategy to use (one of the below defined)
+ *	'partParams'	List of PartitionElems, one for each key column
+ */
+#define PARTITION_STRAT_LIST	'l'
+#define PARTITION_STRAT_RANGE	'r'
+
+typedef struct PartitionBy
+{
+	NodeTag		type;
+	char		strategy;
+	List	   *partParams;
+	int			location;	/* token location, or -1 if unknown */
+} PartitionBy;
+
+/*
+ * PartitionValues - partition values
+ *
+ *	CREATE TABLE ... PARTITION OF ... FOR VALUES (<here>) ...
+ *
+ *	'listvalues'	List of values when defining a list partition
+ *	'rangemins'		If present, min bound of a range partition, one value per column
+ *	'rangemaxs'		If present, max bound of a range partition, one value per column
+ */
+typedef struct PartitionValues
+{
+	NodeTag		type;
+	List	   *listvalues;
+	List	   *rangemins;
+	List	   *rangemaxs;
+	int			location;	/* token location, or -1 if unknown */
+} PartitionValues;
+
+/*
+ * PartitionDef - a single partition definition
+ *
+ *	'name'		Name of this partition
+ *	'parent'	Name of the parent of this partition
+ *	'values'	A PartitionValues struct describing FOR VALUES part of
+ *				the definition
+ */
+typedef struct PartitionDef
+{
+	NodeTag			 type;
+	RangeVar		*name;
+	RangeVar		*parent;
+	PartitionValues	*values;
+} PartitionDef;
 
 /****************************************************************************
  *	Nodes for a Query tree
@@ -1513,7 +1583,15 @@ typedef enum AlterTableType
 	AT_ReplicaIdentity,			/* REPLICA IDENTITY */
 	AT_EnableRowSecurity,		/* ENABLE ROW SECURITY */
 	AT_DisableRowSecurity,		/* DISABLE ROW SECURITY */
-	AT_GenericOptions			/* OPTIONS (...) */
+	AT_GenericOptions,			/* OPTIONS (...) */
+	AT_AttachPartition,			/* CREATE TABLE name PARTITION OF parent
+								 *		FOR VALUES (...)
+								 *  or
+								 * ALTER TABLE parent ATTACH PARTITION name
+								 *		FOR VALUES (...)
+								 *		USING [ TABLE ] table_name */
+	AT_DetachPartition			/* ALTER TABLE parent DETACH PARTITION name
+								 *		[ USING table_name ] */
 } AlterTableType;
 
 typedef struct ReplicaIdentityStmt
@@ -1532,6 +1610,8 @@ typedef struct AlterTableCmd	/* one subcommand of an ALTER TABLE */
 	Node	   *newowner;		/* RoleSpec */
 	Node	   *def;			/* definition of new column, index,
 								 * constraint, or parent table */
+	RangeVar   *using_table;	/* table to use as source/target for
+								 * attach/detach partition sub-commands */
 	DropBehavior behavior;		/* RESTRICT or CASCADE for DROP cases */
 	bool		missing_ok;		/* skip error if missing? */
 } AlterTableCmd;
@@ -1735,9 +1815,13 @@ typedef struct CreateStmt
 {
 	NodeTag		type;
 	RangeVar   *relation;		/* relation to create */
+	RangeVar   *partitionOf;	/* if 'relation' is a partition,
+								 * this is the parent */
 	List	   *tableElts;		/* column definitions (list of ColumnDef) */
 	List	   *inhRelations;	/* relations to inherit from (list of
 								 * inhRelation) */
+	PartitionValues *partValues;	/* partition bounding values */
+	PartitionBy	*partitionby;	/* partition key definition */
 	TypeName   *ofTypename;		/* OF typename */
 	List	   *constraints;	/* constraints (list of Constraint nodes) */
 	List	   *options;		/* options from WITH clause */
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 2414069..c6c290d 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -49,6 +49,7 @@ PG_KEYWORD("assertion", ASSERTION, UNRESERVED_KEYWORD)
 PG_KEYWORD("assignment", ASSIGNMENT, UNRESERVED_KEYWORD)
 PG_KEYWORD("asymmetric", ASYMMETRIC, RESERVED_KEYWORD)
 PG_KEYWORD("at", AT, UNRESERVED_KEYWORD)
+PG_KEYWORD("attach", ATTACH, UNRESERVED_KEYWORD)
 PG_KEYWORD("attribute", ATTRIBUTE, UNRESERVED_KEYWORD)
 PG_KEYWORD("authorization", AUTHORIZATION, TYPE_FUNC_NAME_KEYWORD)
 PG_KEYWORD("backward", BACKWARD, UNRESERVED_KEYWORD)
@@ -126,6 +127,7 @@ PG_KEYWORD("delete", DELETE_P, UNRESERVED_KEYWORD)
 PG_KEYWORD("delimiter", DELIMITER, UNRESERVED_KEYWORD)
 PG_KEYWORD("delimiters", DELIMITERS, UNRESERVED_KEYWORD)
 PG_KEYWORD("desc", DESC, RESERVED_KEYWORD)
+PG_KEYWORD("detach", DETACH, UNRESERVED_KEYWORD)
 PG_KEYWORD("dictionary", DICTIONARY, UNRESERVED_KEYWORD)
 PG_KEYWORD("disable", DISABLE_P, UNRESERVED_KEYWORD)
 PG_KEYWORD("discard", DISCARD, UNRESERVED_KEYWORD)
@@ -226,6 +228,7 @@ PG_KEYWORD("left", LEFT, TYPE_FUNC_NAME_KEYWORD)
 PG_KEYWORD("level", LEVEL, UNRESERVED_KEYWORD)
 PG_KEYWORD("like", LIKE, TYPE_FUNC_NAME_KEYWORD)
 PG_KEYWORD("limit", LIMIT, RESERVED_KEYWORD)
+PG_KEYWORD("list", LIST, UNRESERVED_KEYWORD)
 PG_KEYWORD("listen", LISTEN, UNRESERVED_KEYWORD)
 PG_KEYWORD("load", LOAD, UNRESERVED_KEYWORD)
 PG_KEYWORD("local", LOCAL, UNRESERVED_KEYWORD)
diff --git a/src/include/parser/parse_node.h b/src/include/parser/parse_node.h
index 5249945..aec586d 100644
--- a/src/include/parser/parse_node.h
+++ b/src/include/parser/parse_node.h
@@ -64,7 +64,9 @@ typedef enum ParseExprKind
 	EXPR_KIND_ALTER_COL_TRANSFORM,		/* transform expr in ALTER COLUMN TYPE */
 	EXPR_KIND_EXECUTE_PARAMETER,	/* parameter value in EXECUTE */
 	EXPR_KIND_TRIGGER_WHEN,		/* WHEN condition in CREATE TRIGGER */
-	EXPR_KIND_POLICY			/* USING or WITH CHECK expr in policy */
+	EXPR_KIND_POLICY,			/* USING or WITH CHECK expr in policy */
+	EXPR_KIND_PARTKEY_EXPRESSION, /* partition key expression */
+	EXPR_KIND_PARTITION_VALUES	/* FOR VALUES in CREATE TABLE ... PARTITION OF */
 } ParseExprKind;
 
 
diff --git a/src/include/parser/parse_utilcmd.h b/src/include/parser/parse_utilcmd.h
index d08add3..f871711 100644
--- a/src/include/parser/parse_utilcmd.h
+++ b/src/include/parser/parse_utilcmd.h
@@ -26,4 +26,5 @@ extern void transformRuleStmt(RuleStmt *stmt, const char *queryString,
 				  List **actions, Node **whereClause);
 extern List *transformCreateSchemaStmt(CreateSchemaStmt *stmt);
 
+
 #endif   /* PARSE_UTILCMD_H */
diff --git a/src/include/pg_config_manual.h b/src/include/pg_config_manual.h
index e278fa0..d7badea 100644
--- a/src/include/pg_config_manual.h
+++ b/src/include/pg_config_manual.h
@@ -46,6 +46,12 @@
 #define INDEX_MAX_KEYS		32
 
 /*
+ * Maximum number of columns in a partition key. This is entirely arbitrary
+ * at this point
+ */
+#define PARTITION_MAX_KEYS	16
+
+/*
  * Set the upper and lower bounds of sequence values.
  */
 #define SEQ_MAXVALUE	PG_INT64_MAX
diff --git a/src/include/utils/partition.h b/src/include/utils/partition.h
new file mode 100644
index 0000000..4126f0b
--- /dev/null
+++ b/src/include/utils/partition.h
@@ -0,0 +1,40 @@
+/*-------------------------------------------------------------------------
+ *
+ * partition.h
+ *	  Header file for utility functions related to partitioning
+ *
+ * Copyright (c) 2007-2015, PostgreSQL Global Development Group
+ *
+ * src/include/utils/partition.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTITION_H
+#define PARTITION_H
+
+#include "catalog/pg_partition_fn.h"
+
+typedef struct PartitionKeyTypeInfo
+{
+	Oid		typid[PARTITION_MAX_KEYS];
+	int32	typmod[PARTITION_MAX_KEYS];
+	int16	typlen[PARTITION_MAX_KEYS];
+	bool	typbyval[PARTITION_MAX_KEYS];
+	char	typalign[PARTITION_MAX_KEYS];
+} PartitionKeyTypeInfo;
+
+extern PartitionKeyTypeInfo *get_key_type_info(Relation rel);
+extern int32 range_partition_cmp_max(const void *a, const void *b, void *arg);
+extern int32 range_partition_cmp_min(const void *a, const void *b, void *arg);
+extern int32 range_partition_cmp_bounds(const Datum *lhs, const Datum *rhs,
+								int natts,
+								FmgrInfo *cmpfn);
+extern bool list_partition_overlaps(PartitionBoundInfo *a, PartitionBoundInfo *b,
+								FmgrInfo *cmpfn);
+extern Oid find_partition_for_tuple(Relation rel,
+								PartitionKeyInfo *pkinfo,
+								PartitionInfo *pinfo,
+								TupleTableSlot *slot,
+								EState *estate);
+
+#endif   /* PARTITION_H */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 8a55a09..16118ad 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -18,6 +18,7 @@
 #include "catalog/pg_am.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_index.h"
+#include "catalog/pg_partitioned_rel.h"
 #include "fmgr.h"
 #include "nodes/bitmapset.h"
 #include "rewrite/prs2lock.h"
@@ -63,6 +64,20 @@ typedef struct RelationAmInfo
 	FmgrInfo	amcanreturn;
 } RelationAmInfo;
 
+/* Details of partitions of a partitioned relation */
+typedef struct PartitionInfo
+{
+	int				numpartitions;
+	char			strategy;
+	Oid			   *oids;
+	int			   *listnvalues;
+	Datum		  **listvalues;
+	bool		   *rangeminnull;
+	bool		   *rangemaxnull;
+	Datum		   *rangemins[PARTITION_MAX_KEYS];
+	Datum		   *rangemaxs[PARTITION_MAX_KEYS];
+} PartitionInfo;
+
 /*
  * Here are the contents of a relation cache entry.
  */
@@ -117,6 +132,20 @@ typedef struct RelationData
 	Bitmapset  *rd_keyattr;		/* cols that can be ref'd by foreign keys */
 	Bitmapset  *rd_idattr;		/* included in replica identity index */
 
+	/* Partition key info */
+	Form_pg_partitioned_rel rd_pkey;
+	struct HeapTupleData *rd_pkeytuple;
+
+	MemoryContext rd_pkeycxt;		/* memory context for following items */
+	Oid			*rd_partopfamily;	/* OIDs of op families for each partition col */
+	Oid			*rd_partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo	*rd_partsupfuncs;	/* lookup info for support procedures */
+	List	 	*rd_partexprs;		/* partition key expressions, if any */
+
+	/* Cached info for partitions of this rel */
+	MemoryContext rd_partlistcxt;	/* memory context for the following */
+	PartitionInfo *rd_partitions;	/* Partitions of this relation */
+
 	/*
 	 * rd_options is set whenever rd_rel is loaded into the relcache entry.
 	 * Note that you can NOT look into rd_rel for this data.  NULL means "use
diff --git a/src/include/utils/relcache.h b/src/include/utils/relcache.h
index 6953281..3581db0 100644
--- a/src/include/utils/relcache.h
+++ b/src/include/utils/relcache.h
@@ -43,6 +43,12 @@ extern Oid	RelationGetReplicaIndex(Relation relation);
 extern List *RelationGetIndexExpressions(Relation relation);
 extern List *RelationGetIndexPredicate(Relation relation);
 
+/*
+ * Routines to compute/retrieve cached info of partitioned relations
+ */
+extern List *RelationGetPartitionExpressions(Relation relation);
+extern struct PartitionInfo *RelationGetPartitionInfo(Relation relation);
+
 typedef enum IndexAttrBitmapKind
 {
 	INDEX_ATTR_BITMAP_ALL,
diff --git a/src/include/utils/syscache.h b/src/include/utils/syscache.h
index 18404e2..14bb619 100644
--- a/src/include/utils/syscache.h
+++ b/src/include/utils/syscache.h
@@ -72,6 +72,8 @@ enum SysCacheIdentifier
 	OPEROID,
 	OPFAMILYAMNAMENSP,
 	OPFAMILYOID,
+	PARTITIONID,
+	PARTITIONEDRELID,
 	PROCNAMEARGSNSP,
 	PROCOID,
 	RANGETYPE,
diff --git a/src/test/regress/expected/sanity_check.out b/src/test/regress/expected/sanity_check.out
index eb0bc88..29abb9e 100644
--- a/src/test/regress/expected/sanity_check.out
+++ b/src/test/regress/expected/sanity_check.out
@@ -117,6 +117,8 @@ pg_namespace|t
 pg_opclass|t
 pg_operator|t
 pg_opfamily|t
+pg_partition|t
+pg_partitioned_rel|t
 pg_pltemplate|t
 pg_policy|t
 pg_proc|t
