diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 8314fce78f..4c1d2f607b 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -3827,7 +3827,8 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
the system will be able to skip the scan to validate the implicit
partition constraint. Without such a constraint, the table will be
scanned to validate the partition constraint while holding an
- ACCESS EXCLUSIVE lock on the parent table.
+ ACCESS EXCLUSIVE lock on that partition
+ and a SHARE UPDATE EXCLUSIVE lock on the parent table.
One may then drop the constraint after ATTACH PARTITION
is finished, because it is no longer necessary.
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 5dd6fe02c6..12415b4e99 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -2556,7 +2556,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);
if (cstate->whereClause)
cstate->qualexpr = ExecInitQual(castNode(List, cstate->whereClause),
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 788544ec92..93f13a4778 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3692,6 +3692,9 @@ AlterTableGetLockLevel(List *cmds)
break;
case AT_AttachPartition:
+ cmd_lockmode = ShareUpdateExclusiveLock;
+ break;
+
case AT_DetachPartition:
cmd_lockmode = AccessExclusiveLock;
break;
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 4491ee6991..aaa81f0620 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -167,7 +167,8 @@ static void ExecInitRoutingInfo(ModifyTableState *mtstate,
PartitionDispatch dispatch,
ResultRelInfo *partRelInfo,
int partidx);
-static PartitionDispatch ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute,
+static PartitionDispatch ExecInitPartitionDispatchInfo(EState *estate,
+ PartitionTupleRouting *proute,
Oid partoid, PartitionDispatch parent_pd, int partidx);
static void FormPartitionKeyDatum(PartitionDispatch pd,
TupleTableSlot *slot,
@@ -201,7 +202,8 @@ 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;
@@ -223,7 +225,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
@@ -424,7 +427,8 @@ ExecFindPartition(ModifyTableState *mtstate,
* Create the new PartitionDispatch. We pass the current one
* in as the parent PartitionDispatch
*/
- subdispatch = ExecInitPartitionDispatchInfo(proute,
+ subdispatch = ExecInitPartitionDispatchInfo(mtstate->ps.state,
+ proute,
partdesc->oids[partidx],
dispatch, partidx);
Assert(dispatch->indexes[partidx] >= 0 &&
@@ -988,7 +992,8 @@ ExecInitRoutingInfo(ModifyTableState *mtstate,
* PartitionDispatch later.
*/
static PartitionDispatch
-ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute, Oid partoid,
+ExecInitPartitionDispatchInfo(EState *estate,
+ PartitionTupleRouting *proute, Oid partoid,
PartitionDispatch parent_pd, int partidx)
{
Relation rel;
@@ -997,6 +1002,10 @@ ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute, Oid partoid,
int dispatchidx;
MemoryContext oldcxt;
+ if (estate->es_partition_directory == NULL)
+ estate->es_partition_directory =
+ CreatePartitionDirectory(estate->es_query_cxt);
+
oldcxt = MemoryContextSwitchTo(proute->memcxt);
/*
@@ -1008,7 +1017,7 @@ ExecInitPartitionDispatchInfo(PartitionTupleRouting *proute, Oid partoid,
rel = table_open(partoid, RowExclusiveLock);
else
rel = proute->partition_root;
- partdesc = RelationGetPartitionDesc(rel);
+ partdesc = PartitionDirectoryLookup(estate->es_partition_directory, rel);
pd = (PartitionDispatch) palloc(offsetof(PartitionDispatchData, indexes) +
partdesc->nparts * sizeof(int));
@@ -1554,6 +1563,10 @@ ExecCreatePartitionPruneState(PlanState *planstate,
ListCell *lc;
int i;
+ if (estate->es_partition_directory == NULL)
+ estate->es_partition_directory =
+ CreatePartitionDirectory(estate->es_query_cxt);
+
n_part_hierarchies = list_length(partitionpruneinfo->prune_infos);
Assert(n_part_hierarchies > 0);
@@ -1610,18 +1623,6 @@ ExecCreatePartitionPruneState(PlanState *planstate,
int n_steps;
ListCell *lc3;
- /*
- * We must copy the subplan_map rather than pointing directly to
- * the plan's version, as we may end up making modifications to it
- * later.
- */
- pprune->subplan_map = palloc(sizeof(int) * pinfo->nparts);
- memcpy(pprune->subplan_map, pinfo->subplan_map,
- sizeof(int) * pinfo->nparts);
-
- /* We can use the subpart_map verbatim, since we never modify it */
- pprune->subpart_map = pinfo->subpart_map;
-
/* present_parts is also subject to later modification */
pprune->present_parts = bms_copy(pinfo->present_parts);
@@ -1633,7 +1634,64 @@ ExecCreatePartitionPruneState(PlanState *planstate,
*/
partrel = ExecGetRangeTableRelation(estate, pinfo->rtindex);
partkey = RelationGetPartitionKey(partrel);
- partdesc = RelationGetPartitionDesc(partrel);
+ partdesc = PartitionDirectoryLookup(estate->es_partition_directory,
+ partrel);
+
+ /*
+ * Initialize the subplan_map and subpart_map. Since detaching a
+ * partition requires AccessExclusiveLock, no partitions can have
+ * disappeared, nor can the bounds for any partition have changed.
+ * However, new partitions may have been added.
+ */
+ Assert(partdesc->nparts >= pinfo->nparts);
+ pprune->subplan_map = palloc(sizeof(int) * partdesc->nparts);
+ if (partdesc->nparts == pinfo->nparts)
+ {
+ /*
+ * There are no new partitions, so this is simple. We can
+ * simply point to the subpart_map from the plan, but we must
+ * copy the subplan_map since we may change it later.
+ */
+ pprune->subpart_map = pinfo->subpart_map;
+ memcpy(pprune->subplan_map, pinfo->subplan_map,
+ sizeof(int) * pinfo->nparts);
+
+ /* Double-check that list of relations has not changed. */
+ Assert(memcmp(partdesc->oids, pinfo->relid_map,
+ pinfo->nparts * sizeof(Oid)) == 0);
+ }
+ else
+ {
+ int pd_idx = 0;
+ int pp_idx;
+
+ /*
+ * Some new partitions have appeared since plan time, and
+ * those are reflected in our PartitionDesc but were not
+ * present in the one used to construct subplan_map and
+ * subpart_map. So we must construct new and longer arrays
+ * where the partitions that were originally present map to the
+ * same place, and any added indexes map to -1, as if the
+ * new partitions had been pruned.
+ */
+ pprune->subpart_map = palloc(sizeof(int) * partdesc->nparts);
+ for (pp_idx = 0; pp_idx < partdesc->nparts; ++pp_idx)
+ {
+ if (pinfo->relid_map[pd_idx] != partdesc->oids[pp_idx])
+ {
+ pprune->subplan_map[pp_idx] = -1;
+ pprune->subpart_map[pp_idx] = -1;
+ }
+ else
+ {
+ pprune->subplan_map[pp_idx] =
+ pinfo->subplan_map[pd_idx];
+ pprune->subpart_map[pp_idx] =
+ pinfo->subpart_map[pd_idx++];
+ }
+ }
+ Assert(pd_idx == pinfo->nparts);
+ }
n_steps = list_length(pinfo->pruning_steps);
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 8e9214833d..044d62a56e 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -54,6 +54,7 @@
#include "mb/pg_wchar.h"
#include "nodes/nodeFuncs.h"
#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "storage/lmgr.h"
#include "utils/builtins.h"
#include "utils/memutils.h"
@@ -214,6 +215,13 @@ FreeExecutorState(EState *estate)
estate->es_jit = NULL;
}
+ /* release partition directory, if allocated */
+ if (estate->es_partition_directory)
+ {
+ DestroyPartitionDirectory(estate->es_partition_directory);
+ estate->es_partition_directory = NULL;
+ }
+
/*
* Free the per-query memory context, thereby releasing all working
* memory, including the EState node itself.
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index b9bd86ff8f..a7efe8dcae 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -2186,7 +2186,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/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index a0c1389488..a8a735c247 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -1197,6 +1197,7 @@ _copyPartitionedRelPruneInfo(const PartitionedRelPruneInfo *from)
COPY_SCALAR_FIELD(nexprs);
COPY_POINTER_FIELD(subplan_map, from->nparts * sizeof(int));
COPY_POINTER_FIELD(subpart_map, from->nparts * sizeof(int));
+ COPY_POINTER_FIELD(relid_map, from->nparts * sizeof(int));
COPY_POINTER_FIELD(hasexecparam, from->nexprs * sizeof(bool));
COPY_SCALAR_FIELD(do_initial_prune);
COPY_SCALAR_FIELD(do_exec_prune);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index d9a5e8cb6a..69179a07c3 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -947,6 +947,7 @@ _outPartitionedRelPruneInfo(StringInfo str, const PartitionedRelPruneInfo *node)
WRITE_INT_FIELD(nexprs);
WRITE_INT_ARRAY(subplan_map, node->nparts);
WRITE_INT_ARRAY(subpart_map, node->nparts);
+ WRITE_OID_ARRAY(relid_map, node->nparts);
WRITE_BOOL_ARRAY(hasexecparam, node->nexprs);
WRITE_BOOL_FIELD(do_initial_prune);
WRITE_BOOL_FIELD(do_exec_prune);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 0a34fd9684..4b845b1bb7 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2386,6 +2386,7 @@ _readPartitionedRelPruneInfo(void)
READ_INT_FIELD(nexprs);
READ_INT_ARRAY(subplan_map, local_node->nparts);
READ_INT_ARRAY(subpart_map, local_node->nparts);
+ READ_OID_ARRAY(relid_map, local_node->nparts);
READ_BOOL_ARRAY(hasexecparam, local_node->nexprs);
READ_BOOL_FIELD(do_initial_prune);
READ_BOOL_FIELD(do_exec_prune);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index bc81535905..98dd5281ad 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -56,6 +56,7 @@
#include "parser/analyze.h"
#include "parser/parsetree.h"
#include "parser/parse_agg.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "storage/dsm_impl.h"
#include "utils/rel.h"
@@ -567,6 +568,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
result->jitFlags |= PGJIT_DEFORM;
}
+ if (glob->partition_directory != NULL)
+ DestroyPartitionDirectory(glob->partition_directory);
+
return result;
}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index a014a12060..1fa154e0cb 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -147,6 +147,10 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
{
Assert(rte->relkind == RELKIND_PARTITIONED_TABLE);
+ if (root->glob->partition_directory == NULL)
+ root->glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
/*
* If this table has partitions, recursively expand and lock them.
* While at it, also extract the partition key columns of all the
@@ -246,7 +250,10 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
int i;
RangeTblEntry *childrte;
Index childRTindex;
- PartitionDesc partdesc = RelationGetPartitionDesc(parentrel);
+ PartitionDesc partdesc;
+
+ partdesc = PartitionDirectoryLookup(root->glob->partition_directory,
+ parentrel);
check_stack_depth();
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 78a96b4ee2..30f4dc151b 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -2086,7 +2086,8 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
Assert(relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
- partdesc = RelationGetPartitionDesc(relation);
+ partdesc = PartitionDirectoryLookup(root->glob->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/partdesc.c b/src/backend/partitioning/partdesc.c
index 8a4b63aa26..a4494aca7a 100644
--- a/src/backend/partitioning/partdesc.c
+++ b/src/backend/partitioning/partdesc.c
@@ -14,17 +14,39 @@
#include "postgres.h"
+#include "access/genam.h"
+#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/indexing.h"
#include "catalog/partition.h"
#include "catalog/pg_inherits.h"
#include "partitioning/partbounds.h"
#include "partitioning/partdesc.h"
+#include "storage/bufmgr.h"
+#include "storage/sinval.h"
#include "utils/builtins.h"
+#include "utils/inval.h"
+#include "utils/fmgroids.h"
+#include "utils/hsearch.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/rel.h"
#include "utils/partcache.h"
#include "utils/syscache.h"
+typedef struct PartitionDirectoryData
+{
+ MemoryContext pdir_mcxt;
+ HTAB *pdir_hash;
+} PartitionDirectoryData;
+
+typedef struct PartitionDirectoryEntry
+{
+ Oid reloid;
+ Relation rel;
+ PartitionDesc pd;
+} PartitionDirectoryEntry;
+
/*
* RelationBuildPartitionDesc
* Form rel's partition descriptor
@@ -47,43 +69,93 @@ RelationBuildPartitionDesc(Relation rel)
MemoryContext oldcxt;
int *mapping;
- /* Get partition oids from pg_inherits */
+ /*
+ * Get partition oids from pg_inherits. This uses a single snapshot to
+ * fetch the list of children, so while more children may be getting
+ * added concurrently, whatever this function returns will be accurate
+ * as of some well-defined point in time.
+ */
inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
nparts = list_length(inhoids);
+ /* Allocate arrays for OIDs and boundspecs. */
if (nparts > 0)
{
oids = palloc(nparts * sizeof(Oid));
boundspecs = palloc(nparts * sizeof(PartitionBoundSpec *));
}
- /* Collect bound spec nodes for each partition */
+ /* Collect bound spec nodes for each partition. */
i = 0;
foreach(cell, inhoids)
{
Oid inhrelid = lfirst_oid(cell);
HeapTuple tuple;
- Datum datum;
- bool isnull;
- PartitionBoundSpec *boundspec;
+ PartitionBoundSpec *boundspec = NULL;
+ /* Try fetching the tuple from the catcache, for speed. */
tuple = SearchSysCache1(RELOID, inhrelid);
- if (!HeapTupleIsValid(tuple))
- elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+ if (HeapTupleIsValid(tuple))
+ {
+ Datum datum;
+ bool isnull;
- datum = SysCacheGetAttr(RELOID, tuple,
- Anum_pg_class_relpartbound,
- &isnull);
- if (isnull)
- elog(ERROR, "null relpartbound for relation %u", inhrelid);
- boundspec = stringToNode(TextDatumGetCString(datum));
+ datum = SysCacheGetAttr(RELOID, tuple,
+ Anum_pg_class_relpartbound,
+ &isnull);
+ if (!isnull)
+ boundspec = stringToNode(TextDatumGetCString(datum));
+ ReleaseSysCache(tuple);
+ }
+
+ /*
+ * The system cache may be out of date; if so, we may find no pg_class
+ * tuple or an old one where relpartbound is NULL. In that case, try
+ * the table directly. We can't just AcceptInvalidationMessages() and
+ * retry the system cache lookup because it's possible that a
+ * concurrent ATTACH PARTITION operation has removed itself to the
+ * ProcArray but yet added invalidation messages to the shared queue;
+ * InvalidateSystemCaches() would work, but seems excessive.
+ *
+ * Note that this algorithm assumes that PartitionBoundSpec we manage
+ * to fetch is the right one -- so this is only good enough for
+ * concurrent ATTACH PARTITION, not concurrent DETACH PARTITION
+ * or some hypothetical operation that changes the partition bounds.
+ */
+ if (boundspec == NULL)
+ {
+ Relation pg_class;
+ SysScanDesc scan;
+ ScanKeyData key[1];
+ Datum datum;
+ bool isnull;
+
+ pg_class = table_open(RelationRelationId, AccessShareLock);
+ ScanKeyInit(&key[0],
+ Anum_pg_class_oid,
+ BTEqualStrategyNumber, F_OIDEQ,
+ ObjectIdGetDatum(inhrelid));
+ scan = systable_beginscan(pg_class, ClassOidIndexId, true,
+ NULL, 1, key);
+ tuple = systable_getnext(scan);
+ datum = heap_getattr(tuple, Anum_pg_class_relpartbound,
+ RelationGetDescr(pg_class), &isnull);
+ if (!isnull)
+ boundspec = stringToNode(TextDatumGetCString(datum));
+ systable_endscan(scan);
+ table_close(pg_class, AccessShareLock);
+ }
+
+ /* Sanity checks. */
+ if (!boundspec)
+ elog(ERROR, "missing relpartbound for relation %u", inhrelid);
if (!IsA(boundspec, PartitionBoundSpec))
elog(ERROR, "invalid relpartbound for relation %u", inhrelid);
/*
- * Sanity check: If the PartitionBoundSpec says this is the default
- * partition, its OID should correspond to whatever's stored in
- * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+ * If the PartitionBoundSpec says this is the default partition, its
+ * OID should match pg_partitioned_table.partdefid; if not, the
+ * catalog is corrupt.
*/
if (boundspec->is_default)
{
@@ -95,10 +167,10 @@ RelationBuildPartitionDesc(Relation rel)
inhrelid, partdefid);
}
+ /* Save results. */
oids[i] = inhrelid;
boundspecs[i] = boundspec;
++i;
- ReleaseSysCache(tuple);
}
/* Now build the actual relcache partition descriptor */
@@ -143,13 +215,88 @@ RelationBuildPartitionDesc(Relation rel)
partdesc->oids[index] = oids[i];
/* Record if the partition is a leaf partition */
partdesc->is_leaf[index] =
- (get_rel_relkind(oids[i]) != RELKIND_PARTITIONED_TABLE);
+ (get_rel_relkind(oids[i]) != RELKIND_PARTITIONED_TABLE);
}
MemoryContextSwitchTo(oldcxt);
rel->rd_partdesc = partdesc;
}
+/*
+ * CreatePartitionDirectory
+ * Create a new partition directory object.
+ */
+PartitionDirectory
+CreatePartitionDirectory(MemoryContext mcxt)
+{
+ MemoryContext oldcontext = MemoryContextSwitchTo(mcxt);
+ PartitionDirectory pdir;
+ HASHCTL ctl;
+
+ MemSet(&ctl, 0, sizeof(HASHCTL));
+ ctl.keysize = sizeof(Oid);
+ ctl.entrysize = sizeof(PartitionDirectoryEntry);
+ ctl.hcxt = mcxt;
+
+ pdir = palloc(sizeof(PartitionDirectoryData));
+ pdir->pdir_mcxt = mcxt;
+ pdir->pdir_hash = hash_create("partition directory", 256, &ctl,
+ HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+ MemoryContextSwitchTo(oldcontext);
+ return pdir;
+}
+
+/*
+ * PartitionDirectoryLookup
+ * Look up the partition descriptor for a relation in the directory.
+ *
+ * The purpose of this function is to ensure that we get the same
+ * PartitionDesc for each relation every time we look it up. In the
+ * face of current DDL, different PartitionDescs may be constructed with
+ * different views of the catalog state, but any single particular OID
+ * will always get the same PartitionDesc for as long as the same
+ * PartitionDirectory is used.
+ */
+PartitionDesc
+PartitionDirectoryLookup(PartitionDirectory pdir, Relation rel)
+{
+ PartitionDirectoryEntry *pde;
+ Oid relid = RelationGetRelid(rel);
+ bool found;
+
+ pde = hash_search(pdir->pdir_hash, &relid, HASH_ENTER, &found);
+ if (!found)
+ {
+ /*
+ * We must keep a reference count on the relation so that the
+ * PartitionDesc to which we are pointing can't get destroyed.
+ */
+ RelationIncrementReferenceCount(rel);
+ pde->rel = rel;
+ pde->pd = RelationGetPartitionDesc(rel);
+ Assert(pde->pd != NULL);
+ }
+ return pde->pd;
+}
+
+/*
+ * DestroyPartitionDirectory
+ * Destroy a partition directory.
+ *
+ * Release the reference counts we're holding.
+ */
+void
+DestroyPartitionDirectory(PartitionDirectory pdir)
+{
+ HASH_SEQ_STATUS status;
+ PartitionDirectoryEntry *pde;
+
+ hash_seq_init(&status, pdir->pdir_hash);
+ while ((pde = hash_seq_search(&status)) != NULL)
+ RelationDecrementReferenceCount(pde->rel);
+}
+
/*
* equalPartitionDescs
* Compare two partition descriptors for logical equality
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 8c9721935d..b5c0889935 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -47,8 +47,9 @@
#include "optimizer/appendinfo.h"
#include "optimizer/optimizer.h"
#include "optimizer/pathnode.h"
-#include "partitioning/partprune.h"
+#include "parser/parsetree.h"
#include "partitioning/partbounds.h"
+#include "partitioning/partprune.h"
#include "rewrite/rewriteManip.h"
#include "utils/lsyscache.h"
@@ -359,6 +360,7 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
int partnatts = subpart->part_scheme->partnatts;
int *subplan_map;
int *subpart_map;
+ Oid *relid_map;
List *partprunequal;
List *pruning_steps;
bool contradictory;
@@ -434,6 +436,7 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
*/
subplan_map = (int *) palloc(nparts * sizeof(int));
subpart_map = (int *) palloc(nparts * sizeof(int));
+ relid_map = (Oid *) palloc(nparts * sizeof(int));
present_parts = NULL;
for (i = 0; i < nparts; i++)
@@ -444,6 +447,7 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
subplan_map[i] = subplanidx;
subpart_map[i] = subpartidx;
+ relid_map[i] = planner_rt_fetch(partrel->relid, root)->relid;
if (subplanidx >= 0)
{
present_parts = bms_add_member(present_parts, i);
@@ -462,6 +466,7 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
pinfo->nparts = nparts;
pinfo->subplan_map = subplan_map;
pinfo->subpart_map = subpart_map;
+ pinfo->relid_map = relid_map;
/* Determine which pruning types should be enabled at this level */
doruntimeprune |= analyze_partkey_exprs(pinfo, pruning_steps,
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 0b0508c01d..d9ffb78484 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -2569,6 +2569,26 @@ RelationClearRelation(Relation relation, bool rebuild)
SWAPFIELD(PartitionDesc, rd_partdesc);
SWAPFIELD(MemoryContext, rd_pdcxt);
}
+ else if (rebuild && newrel->rd_pdcxt != NULL)
+ {
+ /*
+ * We are rebuilding a partitioned relation with a non-zero
+ * reference count, so keep the old partition descriptor around,
+ * in case there's a PartitionDirectory with a pointer to it.
+ * Attach it to the new rd_pdcxt so that it gets cleaned up
+ * eventually. In the case where the reference count is 0, this
+ * code is not reached, which should be OK because in that case
+ * there should be no PartitionDirectory with a pointer to the old
+ * entry.
+ *
+ * Note that newrel and relation have already been swapped, so
+ * the "old" partition descriptor is actually the one hanging off
+ * of newrel.
+ */
+ MemoryContextSetParent(newrel->rd_pdcxt, relation->rd_pdcxt);
+ newrel->rd_partdesc = NULL;
+ newrel->rd_pdcxt = NULL;
+ }
#undef SWAPFIELD
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 2048c43c37..b363aba2a5 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -135,7 +135,8 @@ typedef struct PartitionPruneState
PartitionPruningData *partprunedata[FLEXIBLE_ARRAY_MEMBER];
} PartitionPruneState;
-extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
+extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(EState *estate,
+ ModifyTableState *mtstate,
Relation rel);
extern ResultRelInfo *ExecFindPartition(ModifyTableState *mtstate,
ResultRelInfo *rootResultRelInfo,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6a5411eba8..fd13c170d7 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -19,6 +19,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"
@@ -521,6 +522,7 @@ typedef struct EState
*/
ResultRelInfo *es_root_result_relations; /* array of ResultRelInfos */
int es_num_root_result_relations; /* length of the array */
+ PartitionDirectory es_partition_directory; /* for PartitionDesc lookup */
/*
* The following list contains ResultRelInfos created by the tuple routing
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index a008ae07da..7b2cbdbefc 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -144,6 +144,8 @@ typedef struct PlannerGlobal
bool parallelModeNeeded; /* parallel mode actually required? */
char maxParallelHazard; /* worst PROPARALLEL hazard level */
+
+ PartitionDirectory partition_directory; /* partition descriptors */
} PlannerGlobal;
/* macro for fetching the Plan associated with a SubPlan node */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6d087c268f..d66a187a53 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -1108,6 +1108,7 @@ typedef struct PartitionedRelPruneInfo
int nexprs; /* Length of hasexecparam[] */
int *subplan_map; /* subplan index by partition index, or -1 */
int *subpart_map; /* subpart index by partition index, or -1 */
+ Oid *relid_map; /* relation OID by partition index, or -1 */
bool *hasexecparam; /* true if corresponding pruning_step contains
* any PARAM_EXEC Params. */
bool do_initial_prune; /* true if pruning should be performed
diff --git a/src/include/partitioning/partdefs.h b/src/include/partitioning/partdefs.h
index 6e9c128b2c..aec3b3fe63 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/partdesc.h b/src/include/partitioning/partdesc.h
index f72b70dded..da19369e25 100644
--- a/src/include/partitioning/partdesc.h
+++ b/src/include/partitioning/partdesc.h
@@ -31,6 +31,10 @@ typedef struct PartitionDescData
extern void RelationBuildPartitionDesc(Relation rel);
+extern PartitionDirectory CreatePartitionDirectory(MemoryContext mcxt);
+extern PartitionDesc PartitionDirectoryLookup(PartitionDirectory, Relation);
+extern void DestroyPartitionDirectory(PartitionDirectory pdir);
+
extern Oid get_default_oid_from_partdesc(PartitionDesc partdesc);
extern bool equalPartitionDescs(PartitionKey key, PartitionDesc partdesc1,