From b621382a1907f0aa5db5b81fc3be71e2027c3056 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Tue, 18 Dec 2018 18:55:37 -0500
Subject: [PATCH 2/3] Introduce the concept of a partition directory.

Teach the optimizer and executor to use it, so that a single planning
cycle or query execution gets the same PartitionDesc for the same table
every time it looks it up.  This does not prevent changes between
planning and execution, nor does it guarantee that all tables are
expanded according to the same snapshot.
---
 src/backend/commands/copy.c            |  2 +-
 src/backend/executor/execPartition.c   | 32 +++++++++----
 src/backend/executor/nodeModifyTable.c |  2 +-
 src/backend/optimizer/prep/prepunion.c | 88 +++++++++++++++-------------------
 src/backend/optimizer/util/plancat.c   |  6 ++-
 src/backend/partitioning/Makefile      |  2 +-
 src/backend/partitioning/partdir.c     | 76 +++++++++++++++++++++++++++++
 src/backend/utils/cache/relcache.c     | 24 ++++++++++
 src/include/executor/execPartition.h   |  4 +-
 src/include/nodes/execnodes.h          |  4 ++
 src/include/nodes/relation.h           |  4 ++
 src/include/partitioning/partdefs.h    |  2 +
 src/include/partitioning/partdir.h     | 21 ++++++++
 13 files changed, 202 insertions(+), 65 deletions(-)
 create mode 100644 src/backend/partitioning/partdir.c
 create mode 100644 src/include/partitioning/partdir.h

diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 4311e16007..1b69e3c700 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -2528,7 +2528,7 @@ CopyFrom(CopyState cstate)
 	 * CopyFrom tuple routing.
 	 */
 	if (cstate->rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-		proute = ExecSetupPartitionTupleRouting(NULL, cstate->rel);
+		proute = ExecSetupPartitionTupleRouting(estate, NULL, cstate->rel);
 
 	/*
 	 * It's more efficient to prepare a bunch of tuples for insertion, and
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 179a501f30..f10e6fb95c 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -23,6 +23,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "partitioning/partbounds.h"
+#include "partitioning/partdir.h"
 #include "partitioning/partprune.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
@@ -165,8 +166,10 @@ static void ExecInitRoutingInfo(ModifyTableState *mtstate,
 					PartitionDispatch dispatch,
 					ResultRelInfo *partRelInfo,
 					int partidx);
-static PartitionDispatch ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute,
-							  Oid partoid, PartitionDispatch parent_pd, int partidx);
+static PartitionDispatch ExecInitPartitionDispatchInfo(EState *estate,
+							  PartitionTupleRouting *proute,
+							  Oid partoid, PartitionDispatch parent_pd,
+							  int partidx);
 static void FormPartitionKeyDatum(PartitionDispatch pd,
 					  TupleTableSlot *slot,
 					  EState *estate,
@@ -202,7 +205,7 @@ static void find_matching_subplans_recurse(PartitionPruningData *prunedata,
  * it should be estate->es_query_cxt.
  */
 PartitionTupleRouting *
-ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
+ExecSetupPartitionTupleRouting(EState *estate, ModifyTableState *mtstate, Relation rel)
 {
 	PartitionTupleRouting *proute;
 	ModifyTable *node = mtstate ? (ModifyTable *) mtstate->ps.plan : NULL;
@@ -227,7 +230,8 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
 	 * parent as NULL as we don't need to care about any parent of the target
 	 * partitioned table.
 	 */
-	ExecInitPartitionDispatchInfo(proute, RelationGetRelid(rel), NULL, 0);
+	ExecInitPartitionDispatchInfo(estate, proute, RelationGetRelid(rel),
+								  NULL, 0);
 
 	/*
 	 * If performing an UPDATE with tuple routing, we can reuse partition
@@ -428,7 +432,7 @@ ExecFindPartition(ModifyTableState *mtstate,
 				 * Create the new PartitionDispatch.  We pass the current one
 				 * in as the parent PartitionDispatch
 				 */
-				subdispatch = ExecInitPartitionDispatchInfo(proute,
+				subdispatch = ExecInitPartitionDispatchInfo(estate, proute,
 															partdesc->oids[partidx],
 															dispatch, partidx);
 				Assert(dispatch->indexes[partidx] >= 0 &&
@@ -970,8 +974,9 @@ ExecInitRoutingInfo(ModifyTableState *mtstate,
  *		newly created PartitionDispatch later.
  */
 static PartitionDispatch
-ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute, Oid partoid,
-							  PartitionDispatch parent_pd, int partidx)
+ExecInitPartitionDispatchInfo(EState *estate, PartitionTupleRouting *proute,
+							  Oid partoid, PartitionDispatch parent_pd,
+							  int partidx)
 {
 	Relation	rel;
 	PartitionDesc partdesc;
@@ -985,7 +990,12 @@ ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute, Oid partoid,
 		rel = heap_open(partoid, NoLock);
 	else
 		rel = proute->partition_root;
-	partdesc = RelationGetPartitionDesc(rel);
+
+	if (estate->es_partition_directory == NULL)
+		estate->es_partition_directory =
+			CreatePartitionDirectory(estate->es_query_cxt);
+	partdesc = PartitionDirectoryLookup(estate->es_partition_directory,
+										rel);
 
 	pd = (PartitionDispatch) palloc(offsetof(PartitionDispatchData, indexes) +
 									partdesc->nparts * sizeof(int));
@@ -1548,6 +1558,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 	prunestate->do_exec_prune = false;	/* may be set below */
 	prunestate->num_partprunedata = n_part_hierarchies;
 
+	if (estate->es_partition_directory == NULL)
+		estate->es_partition_directory =
+			CreatePartitionDirectory(estate->es_query_cxt);
+
 	/*
 	 * Create a short-term memory context which we'll use when making calls to
 	 * the partition pruning functions.  This avoids possible memory leaks,
@@ -1610,7 +1624,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
 			 */
 			partrel = ExecGetRangeTableRelation(estate, pinfo->rtindex);
 			partkey = RelationGetPartitionKey(partrel);
-			partdesc = RelationGetPartitionDesc(partrel);
+			partdesc = PartitionDirectoryLookup(estate->es_partition_directory, partrel);
 
 			n_steps = list_length(pinfo->pruning_steps);
 
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 3c60bbcd9c..91ffad26f0 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -2229,7 +2229,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE &&
 		(operation == CMD_INSERT || update_tuple_routing_needed))
 		mtstate->mt_partition_tuple_routing =
-			ExecSetupPartitionTupleRouting(mtstate, rel);
+			ExecSetupPartitionTupleRouting(estate, mtstate, rel);
 
 	/*
 	 * Build state for collecting transition tuples.  This requires having a
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index da278f785e..6060302e51 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -48,6 +48,7 @@
 #include "optimizer/tlist.h"
 #include "parser/parse_coerce.h"
 #include "parser/parsetree.h"
+#include "partitioning/partdir.h"
 #include "utils/lsyscache.h"
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
@@ -104,8 +105,7 @@ static void expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte,
 static void expand_partitioned_rtentry(PlannerInfo *root,
 						   RangeTblEntry *parentrte,
 						   Index parentRTindex, Relation parentrel,
-						   PlanRowMark *top_parentrc, LOCKMODE lockmode,
-						   List **appinfos);
+						   PlanRowMark *top_parentrc, List **appinfos);
 static void expand_single_inheritance_child(PlannerInfo *root,
 								RangeTblEntry *parentrte,
 								Index parentRTindex, Relation parentrel,
@@ -1518,7 +1518,6 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	Oid			parentOID;
 	PlanRowMark *oldrc;
 	Relation	oldrelation;
-	LOCKMODE	lockmode;
 	List	   *inhOIDs;
 	ListCell   *l;
 
@@ -1541,37 +1540,13 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	}
 
 	/*
-	 * The rewriter should already have obtained an appropriate lock on each
-	 * relation named in the query.  However, for each child relation we add
-	 * to the query, we must obtain an appropriate lock, because this will be
-	 * the first use of those relations in the parse/rewrite/plan pipeline.
-	 * Child rels should use the same lockmode as their parent.
-	 */
-	lockmode = rte->rellockmode;
-
-	/* Scan for all members of inheritance set, acquire needed locks */
-	inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
-
-	/*
-	 * Check that there's at least one descendant, else treat as no-child
-	 * case.  This could happen despite above has_subclass() check, if table
-	 * once had a child but no longer does.
-	 */
-	if (list_length(inhOIDs) < 2)
-	{
-		/* Clear flag before returning */
-		rte->inh = false;
-		return;
-	}
-
-	/*
-	 * If parent relation is selected FOR UPDATE/SHARE, we need to mark its
-	 * PlanRowMark as isParent = true, and generate a new PlanRowMark for each
-	 * child.
+	 * If parent relation is selected FOR UPDATE/SHARE, we will need to mark
+	 * its PlanRowMark as isParent = true, and generate a new PlanRowMark for
+	 * each child. expand_single_inheritance_child() will handle this, but we
+	 * need to pass down the rowmark for the original parent to make it
+	 * possible.
 	 */
 	oldrc = get_plan_rowmark(root->rowMarks, rti);
-	if (oldrc)
-		oldrc->isParent = true;
 
 	/*
 	 * Must open the parent relation to examine its tupdesc.  We need not lock
@@ -1580,9 +1555,11 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	oldrelation = heap_open(parentOID, NoLock);
 
 	/* Scan the inheritance set and expand it */
-	if (RelationGetPartitionDesc(oldrelation) != NULL)
+	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		Assert(rte->relkind == RELKIND_PARTITIONED_TABLE);
+		/* Create a partition directory unless already done. */
+		if (root->partition_directory == NULL)
+			root->partition_directory = CreatePartitionDirectory(CurrentMemoryContext);
 
 		/*
 		 * If this table has partitions, recursively expand them in the order
@@ -1590,7 +1567,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		 * extract the partition key columns of all the partitioned tables.
 		 */
 		expand_partitioned_rtentry(root, rte, rti, oldrelation, oldrc,
-								   lockmode, &root->append_rel_list);
+								   &root->append_rel_list);
 	}
 	else
 	{
@@ -1598,9 +1575,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		RangeTblEntry *childrte;
 		Index		childRTindex;
 
+		/* Scan for all members of inheritance set, acquire needed locks */
+		inhOIDs = find_all_inheritors(parentOID, rte->rellockmode, NULL);
+
 		/*
-		 * This table has no partitions.  Expand any plain inheritance
-		 * children in the order the OIDs were returned by
+		 * This is not a partitioned table, but it may have plain inheritance
+		 * children.  Expand them in the order that the OIDs were returned by
 		 * find_all_inheritors.
 		 */
 		foreach(l, inhOIDs)
@@ -1622,7 +1602,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 			 */
 			if (childOID != parentOID && RELATION_IS_OTHER_TEMP(newrelation))
 			{
-				heap_close(newrelation, lockmode);
+				heap_close(newrelation, rte->rellockmode);
 				continue;
 			}
 
@@ -1637,11 +1617,11 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		}
 
 		/*
-		 * If all the children were temp tables, pretend it's a
-		 * non-inheritance situation; we don't need Append node in that case.
-		 * The duplicate RTE we added for the parent table is harmless, so we
-		 * don't bother to get rid of it; ditto for the useless PlanRowMark
-		 * node.
+		 * If all the children were temp tables, or there were none, pretend
+		 * it's a non-inheritance situation; we don't need Append node in that
+		 * case.  The duplicate RTE we added for the parent table is harmless,
+		 * so we don't bother to get rid of it; ditto for the useless
+		 * PlanRowMark node.
 		 */
 		if (list_length(appinfos) < 2)
 			rte->inh = false;
@@ -1661,16 +1641,17 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 static void
 expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 						   Index parentRTindex, Relation parentrel,
-						   PlanRowMark *top_parentrc, LOCKMODE lockmode,
-						   List **appinfos)
+						   PlanRowMark *top_parentrc, List **appinfos)
 {
 	int			i;
 	RangeTblEntry *childrte;
 	Index		childRTindex;
-	PartitionDesc partdesc = RelationGetPartitionDesc(parentrel);
+	PartitionDesc partdesc;
 
 	check_stack_depth();
 
+	partdesc = PartitionDirectoryLookup(root->partition_directory, parentrel);
+
 	/* A partitioned table should always have a partition descriptor. */
 	Assert(partdesc);
 
@@ -1707,8 +1688,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 		Oid			childOID = partdesc->oids[i];
 		Relation	childrel;
 
-		/* Open rel; we already have required locks */
-		childrel = heap_open(childOID, NoLock);
+		/* Open and lock child rel */
+		childrel = heap_open(childOID, parentrte->rellockmode);
 
 		/*
 		 * Temporary partitions belonging to other sessions should have been
@@ -1725,8 +1706,7 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
 			expand_partitioned_rtentry(root, childrte, childRTindex,
-									   childrel, top_parentrc, lockmode,
-									   appinfos);
+									   childrel, top_parentrc, appinfos);
 
 		/* Close child relation, but keep locks */
 		heap_close(childrel, NoLock);
@@ -1862,6 +1842,14 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 		/* Include child's rowmark type in top parent's allMarkTypes */
 		top_parentrc->allMarkTypes |= childrc->allMarkTypes;
 
+		/*
+		 * If we create at least one child rowmark, isParent should be set
+		 * on the original rowmark. That's very cheap, so just do it here
+		 * unconditionally without worrying about whether it has been done
+		 * previously.
+		 */
+		top_parentrc->isParent = true;
+
 		root->rowMarks = lappend(root->rowMarks, childrc);
 	}
 }
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index a570ac0aab..f0e5ef070f 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -39,6 +39,7 @@
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
 #include "partitioning/partbounds.h"
+#include "partitioning/partdir.h"
 #include "parser/parse_relation.h"
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
@@ -1903,7 +1904,10 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 
 	Assert(relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
-	partdesc = RelationGetPartitionDesc(relation);
+	/* shouldn't reach here unless expand_inherited_rtentry initialized this */
+	Assert(root->partition_directory != NULL);
+
+	partdesc = PartitionDirectoryLookup(root->partition_directory, relation);
 	partkey = RelationGetPartitionKey(relation);
 	rel->part_scheme = find_partition_scheme(root, relation);
 	Assert(partdesc != NULL && rel->part_scheme != NULL);
diff --git a/src/backend/partitioning/Makefile b/src/backend/partitioning/Makefile
index 278fac3afa..a096b0a0bb 100644
--- a/src/backend/partitioning/Makefile
+++ b/src/backend/partitioning/Makefile
@@ -12,6 +12,6 @@ subdir = src/backend/partitioning
 top_builddir = ../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = partprune.o partbounds.o
+OBJS = partprune.o partbounds.o partdir.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/partitioning/partdir.c b/src/backend/partitioning/partdir.c
new file mode 100644
index 0000000000..463d192f13
--- /dev/null
+++ b/src/backend/partitioning/partdir.c
@@ -0,0 +1,76 @@
+/*-------------------------------------------------------------------------
+ *
+ * partdir.c
+ *		Support for partition directories
+ *
+ * Partition directories provide a mechanism for looking up the
+ * PartitionDesc for a relation in such a way that the answer will be
+ * the same every time the directory is interrogated, even in the face
+ * of concurrent DDL.
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *		  src/backend/partitioning/partdir.c
+ *
+ *-------------------------------------------------------------------------
+*/
+#include "postgres.h"
+
+#include "catalog/pg_class.h"
+#include "partitioning/partdir.h"
+#include "utils/hsearch.h"
+#include "utils/rel.h"
+
+typedef struct PartitionDirectoryData
+{
+	MemoryContext pdir_mcxt;
+	HTAB *pdir_htab;
+} PartitionDirectoryData;
+
+typedef struct PartitionDirectoryEntry
+{
+	Oid	relid;
+	PartitionDesc pd;
+} PartitionDirectoryEntry;
+
+PartitionDirectory
+CreatePartitionDirectory(MemoryContext mcxt)
+{
+	HASHCTL hctl;
+	MemoryContext oldcontext;
+	PartitionDirectory pdir;
+
+	hctl.keysize = sizeof(Oid);
+	hctl.entrysize = sizeof(PartitionDirectoryEntry);
+	hctl.hcxt = mcxt;
+
+	oldcontext = MemoryContextSwitchTo(mcxt);
+
+	pdir = palloc(sizeof(PartitionDirectoryData));
+	pdir->pdir_mcxt = mcxt;
+	pdir->pdir_htab = hash_create("PartitionDirectory", 256, &hctl,
+								  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	MemoryContextSwitchTo(oldcontext);
+	return pdir;
+}
+
+PartitionDesc
+PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
+{
+	PartitionDirectoryEntry *pde;
+	Oid relid;
+	bool found;
+
+	Assert(rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
+	relid = RelationGetRelid(rel);
+	pde = hash_search(pdir->pdir_htab, &relid, HASH_ENTER, &found);
+	if (!found)
+	{
+		pde->pd = RelationGetPartitionDesc(rel);
+		Assert(pde->pd != NULL);
+	}
+	return pde->pd;
+}
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index c3071db1cd..5ec20767de 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -2537,6 +2537,30 @@ RelationClearRelation(Relation relation, bool rebuild)
 			SWAPFIELD(PartitionDesc, rd_partdesc);
 			SWAPFIELD(MemoryContext, rd_pdcxt);
 		}
+		else if (rebuild && newrel->rd_partdesc != NULL)
+		{
+			/*
+			 * If this is a rebuild, that means that the reference count of this
+			 * relation is greater than 0, which means somebody is using it.  We want
+			 * to allow for the possibility that they might still have a pointer to the
+			 * old PartitionDesc, so we don't free it here. Instead, we reparent its
+			 * context under the context for the newly-build PartitionDesc, so that it
+			 * will get freed when that context is eventually destroyed.  While this
+			 * doesn't leak memory permanently, there's no upper limit to how long the
+			 * old PartitionDesc could stick around, so we might want to consider a
+			 * more clever strategy here at some point.  Note also that this strategy
+			 * relies on the fact that a relation which has a partition descriptor
+			 * will never cease having one after a rebuild, which is currently true
+			 * even if the table ends up with no partitions.
+			 *
+			 * NB: At this point in the code, the contents of 'relation' and 'newrel'
+			 * have been swapped and then partially unswapped, so, confusingly, it is
+			 * 'newrel' that points to the old data.
+			 */
+			MemoryContextSetParent(newrel->rd_pdcxt, relation->rd_pdcxt);
+			newrel->rd_pdcxt = NULL;
+			newrel->rd_partdesc = NULL;
+		}
 
 #undef SWAPFIELD
 
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index d3cfb55f9f..17766b1c49 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -135,8 +135,8 @@ typedef struct PartitionPruneState
 	PartitionPruningData *partprunedata[FLEXIBLE_ARRAY_MEMBER];
 } PartitionPruneState;
 
-extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
-							   Relation rel);
+extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(EState *estate,
+							   ModifyTableState *mtstate, Relation rel);
 extern ResultRelInfo *ExecFindPartition(ModifyTableState *mtstate,
 				  ResultRelInfo *rootResultRelInfo,
 				  PartitionTupleRouting *proute,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 5ed0f40f69..985c752d01 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
 #include "lib/pairingheap.h"
 #include "nodes/params.h"
 #include "nodes/plannodes.h"
+#include "partitioning/partdefs.h"
 #include "utils/hsearch.h"
 #include "utils/queryenvironment.h"
 #include "utils/reltrigger.h"
@@ -523,6 +524,9 @@ typedef struct EState
 	 */
 	List	   *es_tuple_routing_result_relations;
 
+	/* Directory of partitions used for any purpose. */
+	PartitionDirectory	es_partition_directory;
+
 	/* Stuff used for firing triggers: */
 	List	   *es_trig_target_relations;	/* trigger-only ResultRelInfos */
 	TupleTableSlot *es_trig_tuple_slot; /* for trigger output tuples */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6fd24203dd..bcf2054838 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -19,6 +19,7 @@
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
+#include "partitioning/partdefs.h"
 #include "storage/block.h"
 
 
@@ -343,6 +344,9 @@ typedef struct PlannerInfo
 
 	/* Does this query modify any partition key columns? */
 	bool		partColsUpdated;
+
+	/* Partition directory. */
+	PartitionDirectory	partition_directory;
 } PlannerInfo;
 
 
diff --git a/src/include/partitioning/partdefs.h b/src/include/partitioning/partdefs.h
index 1fe1b4868e..9d94740d1d 100644
--- a/src/include/partitioning/partdefs.h
+++ b/src/include/partitioning/partdefs.h
@@ -21,4 +21,6 @@ typedef struct PartitionBoundSpec PartitionBoundSpec;
 
 typedef struct PartitionDescData *PartitionDesc;
 
+typedef struct PartitionDirectoryData *PartitionDirectory;
+
 #endif							/* PARTDEFS_H */
diff --git a/src/include/partitioning/partdir.h b/src/include/partitioning/partdir.h
new file mode 100644
index 0000000000..0472575bc1
--- /dev/null
+++ b/src/include/partitioning/partdir.h
@@ -0,0 +1,21 @@
+/*-------------------------------------------------------------------------
+ *
+ * partdir.h
+ *		A partition directory provides stable PartitionDesc lookups
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/partitioning/partdir.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTDIR_H
+#define PARTDIR_H
+
+#include "partitioning/partdefs.h"
+#include "utils/relcache.h"
+
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
+
+#endif							/* PARTDIR_H */
-- 
2.14.3 (Apple Git-98)

