Allow ATTACH PARTITION with only ShareUpdateExclusiveLock.

We still require AccessExclusiveLock on the partition itself, because
otherwise an insert that violates the newly-imposed partition
constraint could be in progress at the same time that we're changing
that constraint; only the lock level on the parent relation is
weakened.

To make this safe, we have to cope with (at least) three separate
problems. First, relevant DDL might commit while we're in the process
of building a PartitionDesc.  If so, find_inheritance_children() might
see a new partition while the RELOID system cache still has the old
partition bound cached, and even before invalidation messages have
been queued.  To fix that, if we see that the pg_class tuple seems to
be missing or to have a null relpartbound, refetch the value directly
from the table. We can't get the wrong value, because DETACH PARTITION
still requires AccessExclusiveLock throughout; if we ever want to
change that, this will need more thought. In testing, I found it quite
difficult to hit even the null-relpartbound case; the race condition
is extremely tight, but the theoretical risk is there.

Second, successive calls to RelationGetPartitionDesc might not return
the same answer.  The query planner will get confused if lookup up the
PartitionDesc for a particular relation does not return a consistent
answer for the entire duration of query planning.  Likewise, query
execution will get confused if the same relation seems to have a
different PartitionDesc at different times.  Invent a new
PartitionDirectory concept and use it to ensure consistency.  This
ensures that a single invocation of either the planner or the executor
sees the same view of the PartitionDesc from beginning to end, but it
does not guarantee that the planner and the executor see the same
view.  Since this allows pointers to old PartitionDesc entries to
survive even after a relcache rebuild, also postpone removing the old
PartitionDesc entry until we're certain no one is using it.

For the most part, it seems to be OK for the planner and executor to
have different views of the PartitionDesc, because the executor will
just ignore any concurrently added partitions which were unknown at
plan time; those partitions won't be part of the inheritance
expansion, but invalidation messages will trigger replanning at some
point.  Normally, this happens by the time the very next command is
executed, but if the next command acquires no locks and executes a
prepared query, it can manage not to notice until a new transaction is
started.  We might want to tighten that up, but it's material for a
separate patch.  There would still be a small window where a query
that started just after an ATTACH PARTITION command committed might
fail to notice its results -- but only if the command starts before
the commit has been acknowledged to the user. All in all, the warts
here around serializability seem small enough to be worth accepting
for the considerable advantage of being able to add partitions without
a full table lock.

Although in general the consequences of new partitions showing up
between planning and execution are limited to the query not noticing
the new partitions, run-time partition pruning will get confused in
that case, so that's the third problem that this patch fixes.
Run-time partition pruning assumes that indexes into the PartitionDesc
are stable between planning and execution.  So, add code so that if
new partitions are added between plan time and execution time, the
indexes stored in the subplan_map[] and subpart_map[] arrays within
the plan's PartitionedRelPruneInfo get adjusted accordingly.  There
does not seem to be a simple way to generalize this scheme to cope
with partitions that are removed, mostly because they could then get
added back again with different bounds, but it works OK for added
partitions.

This code does not try to ensure that every backend participating in
a parallel query sees the same view of the PartitionDesc.  That
currently doesn't matter, because we never pass PartitionDesc
indexes between backends.  Each backend will ignore the concurrently
added partitions which it notices, and it doesn't matter if different
backends are ignoring different sets of concurrently added partitions.
If in the future that matters, for example because we allow writes in
parallel query and want all participants to do tuple routing to the same
set of partitions, the PartitionDirectory concept could be improved to
share PartitionDescs across backends.  There is a draft patch to
serialize and restore PartitionDescs on the thread where this patch
was discussed, which may be a useful place to start.

Patch by me.  Thanks to Alvaro Herrera, David Rowley, Simon Riggs,
Amit Langote, and Michael Paquier for discussion, and to Alvaro
Herrera for some review.

Discussion: http://postgr.es/m/CA+Tgmobt2upbSocvvDej3yzokd7AkiT+PvgFH+a9-5VV1oJNSQ@mail.gmail.com
Discussion: http://postgr.es/m/CA+TgmoZE0r9-cyA-aY6f8WFEROaDLLL7Vf81kZ8MtFCkxpeQSw@mail.gmail.com
Discussion: http://postgr.es/m/CA+TgmoY13KQZF-=HNTrt9UYWYx3_oYOQpu9ioNT49jGgiDpUEA@mail.gmail.com
This commit is contained in:
Robert Haas 2019-03-07 11:13:12 -05:00
parent ec51727f6e
commit 898e5e3290
21 changed files with 313 additions and 44 deletions

View File

@ -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
<literal>ACCESS EXCLUSIVE</literal> lock on the parent table.
<literal>ACCESS EXCLUSIVE</literal> lock on that partition
and a <literal>SHARE UPDATE EXCLUSIVE</literal> lock on the parent table.
One may then drop the constraint after <command>ATTACH PARTITION</command>
is finished, because it is no longer necessary.
</para>

View File

@ -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),

View File

@ -3692,6 +3692,9 @@ AlterTableGetLockLevel(List *cmds)
break;
case AT_AttachPartition:
cmd_lockmode = ShareUpdateExclusiveLock;
break;
case AT_DetachPartition:
cmd_lockmode = AccessExclusiveLock;
break;

View File

@ -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);

View File

@ -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.

View File

@ -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

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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;
}

View File

@ -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();

View File

@ -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);

View File

@ -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

View File

@ -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,

View File

@ -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

View File

@ -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,

View File

@ -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

View File

@ -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 */

View File

@ -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

View File

@ -21,4 +21,6 @@ typedef struct PartitionBoundSpec PartitionBoundSpec;
typedef struct PartitionDescData *PartitionDesc;
typedef struct PartitionDirectoryData *PartitionDirectory;
#endif /* PARTDEFS_H */

View File

@ -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,