Reorganize partitioning code

There's been a massive addition of partitioning code in PostgreSQL 11,
with little oversight on its placement, resulting in a
catalog/partition.c with poorly defined boundaries and responsibilities.
This commit tries to set a couple of distinct modules to separate things
a little bit.  There are no code changes here, only code movement.

There are three new files:
  src/backend/utils/cache/partcache.c
  src/include/partitioning/partdefs.h
  src/include/utils/partcache.h

The previous arrangement of #including catalog/partition.h almost
everywhere is no more.

Authors: Amit Langote and Álvaro Herrera
Discussion: https://postgr.es/m/98e8d509-790a-128c-be7f-e48a5b2d8d97@lab.ntt.co.jp
	https://postgr.es/m/11aa0c50-316b-18bb-722d-c23814f39059@lab.ntt.co.jp
	https://postgr.es/m/143ed9a4-6038-76d4-9a55-502035815e68@lab.ntt.co.jp
	https://postgr.es/m/20180413193503.nynq7bnmgh6vs5vm@alvherre.pgsql
This commit is contained in:
Alvaro Herrera 2018-04-14 21:12:14 -03:00
parent b39fd897e0
commit da6f3e45dd
32 changed files with 3587 additions and 3479 deletions

View File

@ -19,6 +19,7 @@
#include "funcapi.h"
#include "miscadmin.h"
#include "utils/builtins.h"
#include "utils/rel.h"
PG_FUNCTION_INFO_V1(hash_page_type);
PG_FUNCTION_INFO_V1(hash_page_stats);

View File

@ -76,6 +76,7 @@
#include "storage/ipc.h"
#include "storage/spin.h"
#include "tcop/utility.h"
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/memutils.h"

View File

@ -77,6 +77,7 @@
#include "utils/fmgroids.h"
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/ruleutils.h"
#include "utils/snapmgr.h"

File diff suppressed because it is too large Load Diff

View File

@ -18,9 +18,11 @@
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/sysattr.h"
#include "access/tupconvert.h"
#include "catalog/dependency.h"
#include "catalog/indexing.h"
#include "catalog/objectaccess.h"
#include "catalog/partition.h"
#include "catalog/pg_constraint.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_type.h"

View File

@ -56,6 +56,7 @@
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/partcache.h"
#include "utils/regproc.h"
#include "utils/snapmgr.h"
#include "utils/syscache.h"

View File

@ -77,6 +77,7 @@
#include "parser/parse_type.h"
#include "parser/parse_utilcmd.h"
#include "parser/parser.h"
#include "partitioning/partbounds.h"
#include "pgstat.h"
#include "rewrite/rewriteDefine.h"
#include "rewrite/rewriteHandler.h"
@ -92,6 +93,7 @@
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/partcache.h"
#include "utils/relcache.h"
#include "utils/ruleutils.h"
#include "utils/snapmgr.h"

View File

@ -23,6 +23,7 @@
#include "catalog/index.h"
#include "catalog/indexing.h"
#include "catalog/objectaccess.h"
#include "catalog/partition.h"
#include "catalog/pg_constraint.h"
#include "catalog/pg_inherits.h"
#include "catalog/pg_proc.h"

View File

@ -42,7 +42,6 @@
#include "access/transam.h"
#include "access/xact.h"
#include "catalog/namespace.h"
#include "catalog/partition.h"
#include "catalog/pg_publication.h"
#include "commands/matview.h"
#include "commands/trigger.h"
@ -60,6 +59,7 @@
#include "utils/acl.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/partcache.h"
#include "utils/rls.h"
#include "utils/ruleutils.h"
#include "utils/snapmgr.h"

View File

@ -11,9 +11,9 @@
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/partition.h"
#include "catalog/pg_inherits.h"
#include "catalog/pg_type.h"
#include "executor/execPartition.h"
@ -22,10 +22,15 @@
#include "mb/pg_wchar.h"
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "partitioning/partbounds.h"
#include "partitioning/partprune.h"
#include "utils/lsyscache.h"
#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/rls.h"
#include "utils/ruleutils.h"
static PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
int *num_parted, List **leaf_part_oids);
static void get_partition_dispatch_recurse(Relation rel, Relation parent,
@ -35,6 +40,8 @@ static void FormPartitionKeyDatum(PartitionDispatch pd,
EState *estate,
Datum *values,
bool *isnull);
static int get_partition_for_tuple(Relation relation, Datum *values,
bool *isnull);
static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
Datum *values,
bool *isnull,
@ -1022,6 +1029,110 @@ FormPartitionKeyDatum(PartitionDispatch pd,
elog(ERROR, "wrong number of partition key expressions");
}
/*
* get_partition_for_tuple
* Finds partition of relation which accepts the partition key specified
* in values and isnull
*
* Return value is index of the partition (>= 0 and < partdesc->nparts) if one
* found or -1 if none found.
*/
int
get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
{
int bound_offset;
int part_index = -1;
PartitionKey key = RelationGetPartitionKey(relation);
PartitionDesc partdesc = RelationGetPartitionDesc(relation);
/* Route as appropriate based on partitioning strategy. */
switch (key->strategy)
{
case PARTITION_STRATEGY_HASH:
{
PartitionBoundInfo boundinfo = partdesc->boundinfo;
int greatest_modulus = get_hash_partition_greatest_modulus(boundinfo);
uint64 rowHash = compute_hash_value(key->partnatts,
key->partsupfunc,
values, isnull);
part_index = boundinfo->indexes[rowHash % greatest_modulus];
}
break;
case PARTITION_STRATEGY_LIST:
if (isnull[0])
{
if (partition_bound_accepts_nulls(partdesc->boundinfo))
part_index = partdesc->boundinfo->null_index;
}
else
{
bool equal = false;
bound_offset = partition_list_bsearch(key->partsupfunc,
key->partcollation,
partdesc->boundinfo,
values[0], &equal);
if (bound_offset >= 0 && equal)
part_index = partdesc->boundinfo->indexes[bound_offset];
}
break;
case PARTITION_STRATEGY_RANGE:
{
bool equal = false,
range_partkey_has_null = false;
int i;
/*
* No range includes NULL, so this will be accepted by the
* default partition if there is one, and otherwise rejected.
*/
for (i = 0; i < key->partnatts; i++)
{
if (isnull[i])
{
range_partkey_has_null = true;
break;
}
}
if (!range_partkey_has_null)
{
bound_offset = partition_range_datum_bsearch(key->partsupfunc,
key->partcollation,
partdesc->boundinfo,
key->partnatts,
values,
&equal);
/*
* The bound at bound_offset is less than or equal to the
* tuple value, so the bound at offset+1 is the upper
* bound of the partition we're looking for, if there
* actually exists one.
*/
part_index = partdesc->boundinfo->indexes[bound_offset + 1];
}
}
break;
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
}
/*
* part_index < 0 means we failed to find a partition of this parent. Use
* the default partition, if there is one.
*/
if (part_index < 0)
part_index = partdesc->boundinfo->default_index;
return part_index;
}
/*
* ExecBuildSlotPartitionKeyDescription
*

View File

@ -15,12 +15,12 @@
#include "postgres.h"
#include "miscadmin.h"
#include "catalog/partition.h"
#include "optimizer/clauses.h"
#include "optimizer/joininfo.h"
#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "optimizer/prep.h"
#include "partitioning/partbounds.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"

View File

@ -38,6 +38,7 @@
#include "optimizer/plancat.h"
#include "optimizer/predtest.h"
#include "optimizer/prep.h"
#include "partitioning/partbounds.h"
#include "parser/parse_relation.h"
#include "parser/parsetree.h"
#include "rewrite/rewriteManip.h"
@ -45,8 +46,9 @@
#include "storage/bufmgr.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/snapmgr.h"

View File

@ -17,7 +17,6 @@
#include <limits.h>
#include "miscadmin.h"
#include "catalog/partition.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
#include "optimizer/pathnode.h"
@ -27,6 +26,7 @@
#include "optimizer/prep.h"
#include "optimizer/restrictinfo.h"
#include "optimizer/tlist.h"
#include "partitioning/partbounds.h"
#include "utils/hsearch.h"

View File

@ -63,6 +63,7 @@
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/ruleutils.h"
#include "utils/syscache.h"

View File

@ -12,6 +12,6 @@ subdir = src/backend/partitioning
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
OBJS = partprune.o
OBJS = partprune.o partbounds.o
include $(top_srcdir)/src/backend/common.mk

File diff suppressed because it is too large Load Diff

View File

@ -68,6 +68,7 @@
#include "utils/acl.h"
#include "utils/guc.h"
#include "utils/syscache.h"
#include "utils/rel.h"
/* Hook for plugins to get control in ProcessUtility() */

View File

@ -24,7 +24,6 @@
#include "access/sysattr.h"
#include "catalog/dependency.h"
#include "catalog/indexing.h"
#include "catalog/partition.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_am.h"
#include "catalog/pg_authid.h"
@ -64,6 +63,7 @@
#include "utils/guc.h"
#include "utils/hsearch.h"
#include "utils/lsyscache.h"
#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/ruleutils.h"
#include "utils/snapmgr.h"

View File

@ -12,8 +12,8 @@ subdir = src/backend/utils/cache
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = attoptcache.o catcache.o evtcache.o inval.o plancache.o relcache.o \
relmapper.o relfilenodemap.o spccache.o syscache.o lsyscache.o \
typcache.o ts_cache.o
OBJS = attoptcache.o catcache.o evtcache.o inval.o lsyscache.o \
partcache.o plancache.o relcache.o relmapper.o relfilenodemap.o \
spccache.o syscache.o ts_cache.o typcache.o
include $(top_srcdir)/src/backend/common.mk

967
src/backend/utils/cache/partcache.c vendored Normal file
View File

@ -0,0 +1,967 @@
/*-------------------------------------------------------------------------
*
* partcache.c
* Support routines for manipulating partition information cached in
* relcache
*
* Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* src/backend/utils/cache/partcache.c
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
#include "catalog/partition.h"
#include "catalog/pg_inherits.h"
#include "catalog/pg_opclass.h"
#include "catalog/pg_partitioned_table.h"
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/planner.h"
#include "partitioning/partbounds.h"
#include "utils/builtins.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/syscache.h"
static List *generate_partition_qual(Relation rel);
static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
void *arg);
static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
void *arg);
/*
* RelationBuildPartitionKey
* Build and attach to relcache partition key data of relation
*
* Partitioning key data is a complex structure; to avoid complicated logic to
* free individual elements whenever the relcache entry is flushed, we give it
* its own memory context, child of CacheMemoryContext, which can easily be
* deleted on its own. To avoid leaking memory in that context in case of an
* error partway through this function, the context is initially created as a
* child of CurTransactionContext and only re-parented to CacheMemoryContext
* at the end, when no further errors are possible. Also, we don't make this
* context the current context except in very brief code sections, out of fear
* that some of our callees allocate memory on their own which would be leaked
* permanently.
*/
void
RelationBuildPartitionKey(Relation relation)
{
Form_pg_partitioned_table form;
HeapTuple tuple;
bool isnull;
int i;
PartitionKey key;
AttrNumber *attrs;
oidvector *opclass;
oidvector *collation;
ListCell *partexprs_item;
Datum datum;
MemoryContext partkeycxt,
oldcxt;
int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
/*
* The following happens when we have created our pg_class entry but not
* the pg_partitioned_table entry yet.
*/
if (!HeapTupleIsValid(tuple))
return;
partkeycxt = AllocSetContextCreate(CurTransactionContext,
"partition key",
ALLOCSET_SMALL_SIZES);
MemoryContextCopyAndSetIdentifier(partkeycxt,
RelationGetRelationName(relation));
key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
sizeof(PartitionKeyData));
/* Fixed-length attributes */
form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
key->strategy = form->partstrat;
key->partnatts = form->partnatts;
/*
* We can rely on the first variable-length attribute being mapped to the
* relevant field of the catalog's C struct, because all previous
* attributes are non-nullable and fixed-length.
*/
attrs = form->partattrs.values;
/* But use the hard way to retrieve further variable-length attributes */
/* Operator class */
datum = SysCacheGetAttr(PARTRELID, tuple,
Anum_pg_partitioned_table_partclass, &isnull);
Assert(!isnull);
opclass = (oidvector *) DatumGetPointer(datum);
/* Collation */
datum = SysCacheGetAttr(PARTRELID, tuple,
Anum_pg_partitioned_table_partcollation, &isnull);
Assert(!isnull);
collation = (oidvector *) DatumGetPointer(datum);
/* Expressions */
datum = SysCacheGetAttr(PARTRELID, tuple,
Anum_pg_partitioned_table_partexprs, &isnull);
if (!isnull)
{
char *exprString;
Node *expr;
exprString = TextDatumGetCString(datum);
expr = stringToNode(exprString);
pfree(exprString);
/*
* Run the expressions through const-simplification since the planner
* will be comparing them to similarly-processed qual clause operands,
* and may fail to detect valid matches without this step; fix
* opfuncids while at it. We don't need to bother with
* canonicalize_qual() though, because partition expressions should be
* in canonical form already (ie, no need for OR-merging or constant
* elimination).
*/
expr = eval_const_expressions(NULL, expr);
fix_opfuncids(expr);
oldcxt = MemoryContextSwitchTo(partkeycxt);
key->partexprs = (List *) copyObject(expr);
MemoryContextSwitchTo(oldcxt);
}
oldcxt = MemoryContextSwitchTo(partkeycxt);
key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
/* Gather type and collation info as well */
key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
MemoryContextSwitchTo(oldcxt);
/* determine support function number to search for */
procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
HASHEXTENDED_PROC : BTORDER_PROC;
/* Copy partattrs and fill other per-attribute info */
memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
partexprs_item = list_head(key->partexprs);
for (i = 0; i < key->partnatts; i++)
{
AttrNumber attno = key->partattrs[i];
HeapTuple opclasstup;
Form_pg_opclass opclassform;
Oid funcid;
/* Collect opfamily information */
opclasstup = SearchSysCache1(CLAOID,
ObjectIdGetDatum(opclass->values[i]));
if (!HeapTupleIsValid(opclasstup))
elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
key->partopfamily[i] = opclassform->opcfamily;
key->partopcintype[i] = opclassform->opcintype;
/* Get a support function for the specified opfamily and datatypes */
funcid = get_opfamily_proc(opclassform->opcfamily,
opclassform->opcintype,
opclassform->opcintype,
procnum);
if (!OidIsValid(funcid))
ereport(ERROR,
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
NameStr(opclassform->opcname),
(key->strategy == PARTITION_STRATEGY_HASH) ?
"hash" : "btree",
procnum,
format_type_be(opclassform->opcintype))));
fmgr_info_cxt(funcid, &key->partsupfunc[i], partkeycxt);
/* Collation */
key->partcollation[i] = collation->values[i];
/* Collect type information */
if (attno != 0)
{
Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
key->parttypid[i] = att->atttypid;
key->parttypmod[i] = att->atttypmod;
key->parttypcoll[i] = att->attcollation;
}
else
{
if (partexprs_item == NULL)
elog(ERROR, "wrong number of partition key expressions");
key->parttypid[i] = exprType(lfirst(partexprs_item));
key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
partexprs_item = lnext(partexprs_item);
}
get_typlenbyvalalign(key->parttypid[i],
&key->parttyplen[i],
&key->parttypbyval[i],
&key->parttypalign[i]);
ReleaseSysCache(opclasstup);
}
ReleaseSysCache(tuple);
/*
* Success --- reparent our context and make the relcache point to the
* newly constructed key
*/
MemoryContextSetParent(partkeycxt, CacheMemoryContext);
relation->rd_partkeycxt = partkeycxt;
relation->rd_partkey = key;
}
/*
* RelationBuildPartitionDesc
* Form rel's partition descriptor
*
* Not flushed from the cache by RelationClearRelation() unless changed because
* of addition or removal of partition.
*/
void
RelationBuildPartitionDesc(Relation rel)
{
List *inhoids,
*partoids;
Oid *oids = NULL;
List *boundspecs = NIL;
ListCell *cell;
int i,
nparts;
PartitionKey key = RelationGetPartitionKey(rel);
PartitionDesc result;
MemoryContext oldcxt;
int ndatums = 0;
int default_index = -1;
/* Hash partitioning specific */
PartitionHashBound **hbounds = NULL;
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
/* Range partitioning specific */
PartitionRangeBound **rbounds = NULL;
/* Get partition oids from pg_inherits */
inhoids = find_inheritance_children(RelationGetRelid(rel), NoLock);
/* Collect bound spec nodes in a list */
i = 0;
partoids = NIL;
foreach(cell, inhoids)
{
Oid inhrelid = lfirst_oid(cell);
HeapTuple tuple;
Datum datum;
bool isnull;
Node *boundspec;
tuple = SearchSysCache1(RELOID, inhrelid);
if (!HeapTupleIsValid(tuple))
elog(ERROR, "cache lookup failed for relation %u", inhrelid);
/*
* It is possible that the pg_class tuple of a partition has not been
* updated yet to set its relpartbound field. The only case where
* this happens is when we open the parent relation to check using its
* partition descriptor that a new partition's bound does not overlap
* some existing partition.
*/
if (!((Form_pg_class) GETSTRUCT(tuple))->relispartition)
{
ReleaseSysCache(tuple);
continue;
}
datum = SysCacheGetAttr(RELOID, tuple,
Anum_pg_class_relpartbound,
&isnull);
Assert(!isnull);
boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
/*
* Sanity check: If the PartitionBoundSpec says this is the default
* partition, its OID should correspond to whatever's stored in
* pg_partitioned_table.partdefid; if not, the catalog is corrupt.
*/
if (castNode(PartitionBoundSpec, boundspec)->is_default)
{
Oid partdefid;
partdefid = get_default_partition_oid(RelationGetRelid(rel));
if (partdefid != inhrelid)
elog(ERROR, "expected partdefid %u, but got %u",
inhrelid, partdefid);
}
boundspecs = lappend(boundspecs, boundspec);
partoids = lappend_oid(partoids, inhrelid);
ReleaseSysCache(tuple);
}
nparts = list_length(partoids);
if (nparts > 0)
{
oids = (Oid *) palloc(nparts * sizeof(Oid));
i = 0;
foreach(cell, partoids)
oids[i++] = lfirst_oid(cell);
/* Convert from node to the internal representation */
if (key->strategy == PARTITION_STRATEGY_HASH)
{
ndatums = nparts;
hbounds = (PartitionHashBound **)
palloc(nparts * sizeof(PartitionHashBound *));
i = 0;
foreach(cell, boundspecs)
{
PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
lfirst(cell));
if (spec->strategy != PARTITION_STRATEGY_HASH)
elog(ERROR, "invalid strategy in partition bound spec");
hbounds[i] = (PartitionHashBound *)
palloc(sizeof(PartitionHashBound));
hbounds[i]->modulus = spec->modulus;
hbounds[i]->remainder = spec->remainder;
hbounds[i]->index = i;
i++;
}
/* Sort all the bounds in ascending order */
qsort(hbounds, nparts, sizeof(PartitionHashBound *),
qsort_partition_hbound_cmp);
}
else if (key->strategy == PARTITION_STRATEGY_LIST)
{
List *non_null_values = NIL;
/*
* Create a unified list of non-null values across all partitions.
*/
i = 0;
null_index = -1;
foreach(cell, boundspecs)
{
PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
lfirst(cell));
ListCell *c;
if (spec->strategy != PARTITION_STRATEGY_LIST)
elog(ERROR, "invalid strategy in partition bound spec");
/*
* Note the index of the partition bound spec for the default
* partition. There's no datum to add to the list of non-null
* datums for this partition.
*/
if (spec->is_default)
{
default_index = i;
i++;
continue;
}
foreach(c, spec->listdatums)
{
Const *val = castNode(Const, lfirst(c));
PartitionListValue *list_value = NULL;
if (!val->constisnull)
{
list_value = (PartitionListValue *)
palloc0(sizeof(PartitionListValue));
list_value->index = i;
list_value->value = val->constvalue;
}
else
{
/*
* Never put a null into the values array, flag
* instead for the code further down below where we
* construct the actual relcache struct.
*/
if (null_index != -1)
elog(ERROR, "found null more than once");
null_index = i;
}
if (list_value)
non_null_values = lappend(non_null_values,
list_value);
}
i++;
}
ndatums = list_length(non_null_values);
/*
* Collect all list values in one array. Alongside the value, we
* also save the index of partition the value comes from.
*/
all_values = (PartitionListValue **) palloc(ndatums *
sizeof(PartitionListValue *));
i = 0;
foreach(cell, non_null_values)
{
PartitionListValue *src = lfirst(cell);
all_values[i] = (PartitionListValue *)
palloc(sizeof(PartitionListValue));
all_values[i]->value = src->value;
all_values[i]->index = src->index;
i++;
}
qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
qsort_partition_list_value_cmp, (void *) key);
}
else if (key->strategy == PARTITION_STRATEGY_RANGE)
{
int k;
PartitionRangeBound **all_bounds,
*prev;
all_bounds = (PartitionRangeBound **) palloc0(2 * nparts *
sizeof(PartitionRangeBound *));
/*
* Create a unified list of range bounds across all the
* partitions.
*/
i = ndatums = 0;
foreach(cell, boundspecs)
{
PartitionBoundSpec *spec = castNode(PartitionBoundSpec,
lfirst(cell));
PartitionRangeBound *lower,
*upper;
if (spec->strategy != PARTITION_STRATEGY_RANGE)
elog(ERROR, "invalid strategy in partition bound spec");
/*
* Note the index of the partition bound spec for the default
* partition. There's no datum to add to the allbounds array
* for this partition.
*/
if (spec->is_default)
{
default_index = i++;
continue;
}
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
upper = make_one_range_bound(key, i, spec->upperdatums,
false);
all_bounds[ndatums++] = lower;
all_bounds[ndatums++] = upper;
i++;
}
Assert(ndatums == nparts * 2 ||
(default_index != -1 && ndatums == (nparts - 1) * 2));
/* Sort all the bounds in ascending order */
qsort_arg(all_bounds, ndatums,
sizeof(PartitionRangeBound *),
qsort_partition_rbound_cmp,
(void *) key);
/* Save distinct bounds from all_bounds into rbounds. */
rbounds = (PartitionRangeBound **)
palloc(ndatums * sizeof(PartitionRangeBound *));
k = 0;
prev = NULL;
for (i = 0; i < ndatums; i++)
{
PartitionRangeBound *cur = all_bounds[i];
bool is_distinct = false;
int j;
/* Is the current bound distinct from the previous one? */
for (j = 0; j < key->partnatts; j++)
{
Datum cmpval;
if (prev == NULL || cur->kind[j] != prev->kind[j])
{
is_distinct = true;
break;
}
/*
* If the bounds are both MINVALUE or MAXVALUE, stop now
* and treat them as equal, since any values after this
* point must be ignored.
*/
if (cur->kind[j] != PARTITION_RANGE_DATUM_VALUE)
break;
cmpval = FunctionCall2Coll(&key->partsupfunc[j],
key->partcollation[j],
cur->datums[j],
prev->datums[j]);
if (DatumGetInt32(cmpval) != 0)
{
is_distinct = true;
break;
}
}
/*
* Only if the bound is distinct save it into a temporary
* array i.e. rbounds which is later copied into boundinfo
* datums array.
*/
if (is_distinct)
rbounds[k++] = all_bounds[i];
prev = cur;
}
/* Update ndatums to hold the count of distinct datums. */
ndatums = k;
}
else
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
}
/* Now build the actual relcache partition descriptor */
rel->rd_pdcxt = AllocSetContextCreate(CacheMemoryContext,
"partition descriptor",
ALLOCSET_DEFAULT_SIZES);
MemoryContextCopyAndSetIdentifier(rel->rd_pdcxt, RelationGetRelationName(rel));
oldcxt = MemoryContextSwitchTo(rel->rd_pdcxt);
result = (PartitionDescData *) palloc0(sizeof(PartitionDescData));
result->nparts = nparts;
if (nparts > 0)
{
PartitionBoundInfo boundinfo;
int *mapping;
int next_index = 0;
result->oids = (Oid *) palloc0(nparts * sizeof(Oid));
boundinfo = (PartitionBoundInfoData *)
palloc0(sizeof(PartitionBoundInfoData));
boundinfo->strategy = key->strategy;
boundinfo->default_index = -1;
boundinfo->ndatums = ndatums;
boundinfo->null_index = -1;
boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
/* Initialize mapping array with invalid values */
mapping = (int *) palloc(sizeof(int) * nparts);
for (i = 0; i < nparts; i++)
mapping[i] = -1;
switch (key->strategy)
{
case PARTITION_STRATEGY_HASH:
{
/* Modulus are stored in ascending order */
int greatest_modulus = hbounds[ndatums - 1]->modulus;
boundinfo->indexes = (int *) palloc(greatest_modulus *
sizeof(int));
for (i = 0; i < greatest_modulus; i++)
boundinfo->indexes[i] = -1;
for (i = 0; i < nparts; i++)
{
int modulus = hbounds[i]->modulus;
int remainder = hbounds[i]->remainder;
boundinfo->datums[i] = (Datum *) palloc(2 *
sizeof(Datum));
boundinfo->datums[i][0] = Int32GetDatum(modulus);
boundinfo->datums[i][1] = Int32GetDatum(remainder);
while (remainder < greatest_modulus)
{
/* overlap? */
Assert(boundinfo->indexes[remainder] == -1);
boundinfo->indexes[remainder] = i;
remainder += modulus;
}
mapping[hbounds[i]->index] = i;
pfree(hbounds[i]);
}
pfree(hbounds);
break;
}
case PARTITION_STRATEGY_LIST:
{
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
/*
* Copy values. Indexes of individual values are mapped
* to canonical values so that they match for any two list
* partitioned tables with same number of partitions and
* same lists per partition. One way to canonicalize is
* to assign the index in all_values[] of the smallest
* value of each partition, as the index of all of the
* partition's values.
*/
for (i = 0; i < ndatums; i++)
{
boundinfo->datums[i] = (Datum *) palloc(sizeof(Datum));
boundinfo->datums[i][0] = datumCopy(all_values[i]->value,
key->parttypbyval[0],
key->parttyplen[0]);
/* If the old index has no mapping, assign one */
if (mapping[all_values[i]->index] == -1)
mapping[all_values[i]->index] = next_index++;
boundinfo->indexes[i] = mapping[all_values[i]->index];
}
/*
* If null-accepting partition has no mapped index yet,
* assign one. This could happen if such partition
* accepts only null and hence not covered in the above
* loop which only handled non-null values.
*/
if (null_index != -1)
{
Assert(null_index >= 0);
if (mapping[null_index] == -1)
mapping[null_index] = next_index++;
boundinfo->null_index = mapping[null_index];
}
/* Assign mapped index for the default partition. */
if (default_index != -1)
{
/*
* The default partition accepts any value not
* specified in the lists of other partitions, hence
* it should not get mapped index while assigning
* those for non-null datums.
*/
Assert(default_index >= 0 &&
mapping[default_index] == -1);
mapping[default_index] = next_index++;
boundinfo->default_index = mapping[default_index];
}
/* All partition must now have a valid mapping */
Assert(next_index == nparts);
break;
}
case PARTITION_STRATEGY_RANGE:
{
boundinfo->kind = (PartitionRangeDatumKind **)
palloc(ndatums *
sizeof(PartitionRangeDatumKind *));
boundinfo->indexes = (int *) palloc((ndatums + 1) *
sizeof(int));
for (i = 0; i < ndatums; i++)
{
int j;
boundinfo->datums[i] = (Datum *) palloc(key->partnatts *
sizeof(Datum));
boundinfo->kind[i] = (PartitionRangeDatumKind *)
palloc(key->partnatts *
sizeof(PartitionRangeDatumKind));
for (j = 0; j < key->partnatts; j++)
{
if (rbounds[i]->kind[j] == PARTITION_RANGE_DATUM_VALUE)
boundinfo->datums[i][j] =
datumCopy(rbounds[i]->datums[j],
key->parttypbyval[j],
key->parttyplen[j]);
boundinfo->kind[i][j] = rbounds[i]->kind[j];
}
/*
* There is no mapping for invalid indexes.
*
* Any lower bounds in the rbounds array have invalid
* indexes assigned, because the values between the
* previous bound (if there is one) and this (lower)
* bound are not part of the range of any existing
* partition.
*/
if (rbounds[i]->lower)
boundinfo->indexes[i] = -1;
else
{
int orig_index = rbounds[i]->index;
/* If the old index has no mapping, assign one */
if (mapping[orig_index] == -1)
mapping[orig_index] = next_index++;
boundinfo->indexes[i] = mapping[orig_index];
}
}
/* Assign mapped index for the default partition. */
if (default_index != -1)
{
Assert(default_index >= 0 && mapping[default_index] == -1);
mapping[default_index] = next_index++;
boundinfo->default_index = mapping[default_index];
}
boundinfo->indexes[i] = -1;
break;
}
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
}
result->boundinfo = boundinfo;
/*
* Now assign OIDs from the original array into mapped indexes of the
* result array. Order of OIDs in the former is defined by the
* catalog scan that retrieved them, whereas that in the latter is
* defined by canonicalized representation of the partition bounds.
*/
for (i = 0; i < nparts; i++)
result->oids[mapping[i]] = oids[i];
pfree(mapping);
}
MemoryContextSwitchTo(oldcxt);
rel->rd_partdesc = result;
}
/*
* RelationGetPartitionQual
*
* Returns a list of partition quals
*/
List *
RelationGetPartitionQual(Relation rel)
{
/* Quick exit */
if (!rel->rd_rel->relispartition)
return NIL;
return generate_partition_qual(rel);
}
/*
* get_partition_qual_relid
*
* Returns an expression tree describing the passed-in relation's partition
* constraint. If there is no partition constraint returns NULL; this can
* happen if the default partition is the only partition.
*/
Expr *
get_partition_qual_relid(Oid relid)
{
Relation rel = heap_open(relid, AccessShareLock);
Expr *result = NULL;
List *and_args;
/* Do the work only if this relation is a partition. */
if (rel->rd_rel->relispartition)
{
and_args = generate_partition_qual(rel);
if (and_args == NIL)
result = NULL;
else if (list_length(and_args) > 1)
result = makeBoolExpr(AND_EXPR, and_args, -1);
else
result = linitial(and_args);
}
/* Keep the lock. */
heap_close(rel, NoLock);
return result;
}
/*
* generate_partition_qual
*
* Generate partition predicate from rel's partition bound expression. The
* function returns a NIL list if there is no predicate.
*
* Result expression tree is stored CacheMemoryContext to ensure it survives
* as long as the relcache entry. But we should be running in a less long-lived
* working context. To avoid leaking cache memory if this routine fails partway
* through, we build in working memory and then copy the completed structure
* into cache memory.
*/
static List *
generate_partition_qual(Relation rel)
{
HeapTuple tuple;
MemoryContext oldcxt;
Datum boundDatum;
bool isnull;
PartitionBoundSpec *bound;
List *my_qual = NIL,
*result = NIL;
Relation parent;
bool found_whole_row;
/* Guard against stack overflow due to overly deep partition tree */
check_stack_depth();
/* Quick copy */
if (rel->rd_partcheck != NIL)
return copyObject(rel->rd_partcheck);
/* Grab at least an AccessShareLock on the parent table */
parent = heap_open(get_partition_parent(RelationGetRelid(rel)),
AccessShareLock);
/* Get pg_class.relpartbound */
tuple = SearchSysCache1(RELOID, RelationGetRelid(rel));
if (!HeapTupleIsValid(tuple))
elog(ERROR, "cache lookup failed for relation %u",
RelationGetRelid(rel));
boundDatum = SysCacheGetAttr(RELOID, tuple,
Anum_pg_class_relpartbound,
&isnull);
if (isnull) /* should not happen */
elog(ERROR, "relation \"%s\" has relpartbound = null",
RelationGetRelationName(rel));
bound = castNode(PartitionBoundSpec,
stringToNode(TextDatumGetCString(boundDatum)));
ReleaseSysCache(tuple);
my_qual = get_qual_from_partbound(rel, parent, bound);
/* Add the parent's quals to the list (if any) */
if (parent->rd_rel->relispartition)
result = list_concat(generate_partition_qual(parent), my_qual);
else
result = my_qual;
/*
* Change Vars to have partition's attnos instead of the parent's. We do
* this after we concatenate the parent's quals, because we want every Var
* in it to bear this relation's attnos. It's safe to assume varno = 1
* here.
*/
result = map_partition_varattnos(result, 1, rel, parent,
&found_whole_row);
/* There can never be a whole-row reference here */
if (found_whole_row)
elog(ERROR, "unexpected whole-row reference found in partition key");
/* Save a copy in the relcache */
oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
rel->rd_partcheck = copyObject(result);
MemoryContextSwitchTo(oldcxt);
/* Keep the parent locked until commit */
heap_close(parent, NoLock);
return result;
}
/*
* qsort_partition_hbound_cmp
*
* We sort hash bounds by modulus, then by remainder.
*/
static int32
qsort_partition_hbound_cmp(const void *a, const void *b)
{
PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
return partition_hbound_cmp(h1->modulus, h1->remainder,
h2->modulus, h2->remainder);
}
/*
* qsort_partition_list_value_cmp
*
* Compare two list partition bound datums
*/
static int32
qsort_partition_list_value_cmp(const void *a, const void *b, void *arg)
{
Datum val1 = (*(const PartitionListValue **) a)->value,
val2 = (*(const PartitionListValue **) b)->value;
PartitionKey key = (PartitionKey) arg;
return DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
val1, val2));
}
/* Used when sorting range bounds across all range partitions */
static int32
qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
{
PartitionRangeBound *b1 = (*(PartitionRangeBound *const *) a);
PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
PartitionKey key = (PartitionKey) arg;
return partition_rbound_cmp(key->partnatts, key->partsupfunc,
key->partcollation, b1->datums, b1->kind,
b1->lower, b2);
}

View File

@ -73,6 +73,7 @@
#include "optimizer/cost.h"
#include "optimizer/prep.h"
#include "optimizer/var.h"
#include "partitioning/partbounds.h"
#include "pgstat.h"
#include "rewrite/rewriteDefine.h"
#include "rewrite/rowsecurity.h"
@ -85,6 +86,7 @@
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/partcache.h"
#include "utils/relmapper.h"
#include "utils/resowner_private.h"
#include "utils/snapmgr.h"
@ -265,7 +267,6 @@ static HeapTuple ScanPgRelation(Oid targetRelId, bool indexOK, bool force_non_hi
static Relation AllocateRelationDesc(Form_pg_class relp);
static void RelationParseRelOptions(Relation relation, HeapTuple tuple);
static void RelationBuildTupleDesc(Relation relation);
static void RelationBuildPartitionKey(Relation relation);
static Relation RelationBuildDesc(Oid targetRelId, bool insertIt);
static void RelationInitPhysicalAddr(Relation relation);
static void load_critical_index(Oid indexoid, Oid heapoid);
@ -873,211 +874,6 @@ RelationBuildRuleLock(Relation relation)
relation->rd_rules = rulelock;
}
/*
* RelationBuildPartitionKey
* Build and attach to relcache partition key data of relation
*
* Partitioning key data is a complex structure; to avoid complicated logic to
* free individual elements whenever the relcache entry is flushed, we give it
* its own memory context, child of CacheMemoryContext, which can easily be
* deleted on its own. To avoid leaking memory in that context in case of an
* error partway through this function, the context is initially created as a
* child of CurTransactionContext and only re-parented to CacheMemoryContext
* at the end, when no further errors are possible. Also, we don't make this
* context the current context except in very brief code sections, out of fear
* that some of our callees allocate memory on their own which would be leaked
* permanently.
*/
static void
RelationBuildPartitionKey(Relation relation)
{
Form_pg_partitioned_table form;
HeapTuple tuple;
bool isnull;
int i;
PartitionKey key;
AttrNumber *attrs;
oidvector *opclass;
oidvector *collation;
ListCell *partexprs_item;
Datum datum;
MemoryContext partkeycxt,
oldcxt;
int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
/*
* The following happens when we have created our pg_class entry but not
* the pg_partitioned_table entry yet.
*/
if (!HeapTupleIsValid(tuple))
return;
partkeycxt = AllocSetContextCreate(CurTransactionContext,
"partition key",
ALLOCSET_SMALL_SIZES);
MemoryContextCopyAndSetIdentifier(partkeycxt,
RelationGetRelationName(relation));
key = (PartitionKey) MemoryContextAllocZero(partkeycxt,
sizeof(PartitionKeyData));
/* Fixed-length attributes */
form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
key->strategy = form->partstrat;
key->partnatts = form->partnatts;
/*
* We can rely on the first variable-length attribute being mapped to the
* relevant field of the catalog's C struct, because all previous
* attributes are non-nullable and fixed-length.
*/
attrs = form->partattrs.values;
/* But use the hard way to retrieve further variable-length attributes */
/* Operator class */
datum = SysCacheGetAttr(PARTRELID, tuple,
Anum_pg_partitioned_table_partclass, &isnull);
Assert(!isnull);
opclass = (oidvector *) DatumGetPointer(datum);
/* Collation */
datum = SysCacheGetAttr(PARTRELID, tuple,
Anum_pg_partitioned_table_partcollation, &isnull);
Assert(!isnull);
collation = (oidvector *) DatumGetPointer(datum);
/* Expressions */
datum = SysCacheGetAttr(PARTRELID, tuple,
Anum_pg_partitioned_table_partexprs, &isnull);
if (!isnull)
{
char *exprString;
Node *expr;
exprString = TextDatumGetCString(datum);
expr = stringToNode(exprString);
pfree(exprString);
/*
* Run the expressions through const-simplification since the planner
* will be comparing them to similarly-processed qual clause operands,
* and may fail to detect valid matches without this step; fix
* opfuncids while at it. We don't need to bother with
* canonicalize_qual() though, because partition expressions should be
* in canonical form already (ie, no need for OR-merging or constant
* elimination).
*/
expr = eval_const_expressions(NULL, expr);
fix_opfuncids(expr);
oldcxt = MemoryContextSwitchTo(partkeycxt);
key->partexprs = (List *) copyObject(expr);
MemoryContextSwitchTo(oldcxt);
}
oldcxt = MemoryContextSwitchTo(partkeycxt);
key->partattrs = (AttrNumber *) palloc0(key->partnatts * sizeof(AttrNumber));
key->partopfamily = (Oid *) palloc0(key->partnatts * sizeof(Oid));
key->partopcintype = (Oid *) palloc0(key->partnatts * sizeof(Oid));
key->partsupfunc = (FmgrInfo *) palloc0(key->partnatts * sizeof(FmgrInfo));
key->partcollation = (Oid *) palloc0(key->partnatts * sizeof(Oid));
/* Gather type and collation info as well */
key->parttypid = (Oid *) palloc0(key->partnatts * sizeof(Oid));
key->parttypmod = (int32 *) palloc0(key->partnatts * sizeof(int32));
key->parttyplen = (int16 *) palloc0(key->partnatts * sizeof(int16));
key->parttypbyval = (bool *) palloc0(key->partnatts * sizeof(bool));
key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
MemoryContextSwitchTo(oldcxt);
/* determine support function number to search for */
procnum = (key->strategy == PARTITION_STRATEGY_HASH) ?
HASHEXTENDED_PROC : BTORDER_PROC;
/* Copy partattrs and fill other per-attribute info */
memcpy(key->partattrs, attrs, key->partnatts * sizeof(int16));
partexprs_item = list_head(key->partexprs);
for (i = 0; i < key->partnatts; i++)
{
AttrNumber attno = key->partattrs[i];
HeapTuple opclasstup;
Form_pg_opclass opclassform;
Oid funcid;
/* Collect opfamily information */
opclasstup = SearchSysCache1(CLAOID,
ObjectIdGetDatum(opclass->values[i]));
if (!HeapTupleIsValid(opclasstup))
elog(ERROR, "cache lookup failed for opclass %u", opclass->values[i]);
opclassform = (Form_pg_opclass) GETSTRUCT(opclasstup);
key->partopfamily[i] = opclassform->opcfamily;
key->partopcintype[i] = opclassform->opcintype;
/* Get a support function for the specified opfamily and datatypes */
funcid = get_opfamily_proc(opclassform->opcfamily,
opclassform->opcintype,
opclassform->opcintype,
procnum);
if (!OidIsValid(funcid))
ereport(ERROR,
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
errmsg("operator class \"%s\" of access method %s is missing support function %d for type %s",
NameStr(opclassform->opcname),
(key->strategy == PARTITION_STRATEGY_HASH) ?
"hash" : "btree",
procnum,
format_type_be(opclassform->opcintype))));
fmgr_info_cxt(funcid, &key->partsupfunc[i], partkeycxt);
/* Collation */
key->partcollation[i] = collation->values[i];
/* Collect type information */
if (attno != 0)
{
Form_pg_attribute att = TupleDescAttr(relation->rd_att, attno - 1);
key->parttypid[i] = att->atttypid;
key->parttypmod[i] = att->atttypmod;
key->parttypcoll[i] = att->attcollation;
}
else
{
if (partexprs_item == NULL)
elog(ERROR, "wrong number of partition key expressions");
key->parttypid[i] = exprType(lfirst(partexprs_item));
key->parttypmod[i] = exprTypmod(lfirst(partexprs_item));
key->parttypcoll[i] = exprCollation(lfirst(partexprs_item));
partexprs_item = lnext(partexprs_item);
}
get_typlenbyvalalign(key->parttypid[i],
&key->parttyplen[i],
&key->parttypbyval[i],
&key->parttypalign[i]);
ReleaseSysCache(opclasstup);
}
ReleaseSysCache(tuple);
/*
* Success --- reparent our context and make the relcache point to the
* newly constructed key
*/
MemoryContextSetParent(partkeycxt, CacheMemoryContext);
relation->rd_partkeycxt = partkeycxt;
relation->rd_partkey = key;
}
/*
* equalRuleLocks
*

View File

@ -15,13 +15,14 @@
#include "postgres.h"
#include "funcapi.h"
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xlog_internal.h"
#include "access/xlog.h"
#include "catalog/pg_control.h"
#include "catalog/pg_type.h"
#include "common/controldata_utils.h"
#include "funcapi.h"
#include "miscadmin.h"
#include "utils/builtins.h"
#include "utils/pg_lsn.h"
#include "utils/timestamp.h"

View File

@ -14,22 +14,12 @@
#define PARTITION_H
#include "fmgr.h"
#include "executor/tuptable.h"
#include "nodes/execnodes.h"
#include "parser/parse_node.h"
#include "utils/rel.h"
#include "partitioning/partdefs.h"
#include "utils/relcache.h"
/* Seed for the extended hash function */
#define HASH_PARTITION_SEED UINT64CONST(0x7A5B22367996DCFD)
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
*
* The internal structure appears in partbounds.h.
*/
typedef struct PartitionBoundInfoData *PartitionBoundInfo;
/*
* Information about partitions of a partitioned table.
*/
@ -40,37 +30,17 @@ typedef struct PartitionDescData
PartitionBoundInfo boundinfo; /* collection of partition bounds */
} PartitionDescData;
typedef struct PartitionDescData *PartitionDesc;
extern void RelationBuildPartitionDesc(Relation relation);
extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
bool *parttypbyval, PartitionBoundInfo b1,
PartitionBoundInfo b2);
extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
PartitionKey key);
extern void check_new_partition_bound(char *relname, Relation parent,
PartitionBoundSpec *spec);
extern Oid get_partition_parent(Oid relid);
extern List *get_partition_ancestors(Oid relid);
extern List *get_qual_from_partbound(Relation rel, Relation parent,
PartitionBoundSpec *spec);
extern List *map_partition_varattnos(List *expr, int fromrel_varno,
Relation to_rel, Relation from_rel,
bool *found_whole_row);
extern List *RelationGetPartitionQual(Relation rel);
extern Expr *get_partition_qual_relid(Oid relid);
extern bool has_partition_attrs(Relation rel, Bitmapset *attnums,
bool *used_in_expr);
extern Oid get_default_oid_from_partdesc(PartitionDesc partdesc);
extern Oid get_default_partition_oid(Oid parentId);
extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
extern void check_default_allows_bound(Relation parent, Relation defaultRel,
PartitionBoundSpec *new_spec);
extern List *get_proposed_default_constraint(List *new_part_constaints);
extern int get_partition_for_tuple(Relation relation, Datum *values,
bool *isnull);
#endif /* PARTITION_H */

View File

@ -18,7 +18,6 @@
#include "catalog/dependency.h"
#include "catalog/objectaddress.h"
#include "nodes/parsenodes.h"
#include "catalog/partition.h"
#include "storage/lock.h"
#include "utils/relcache.h"

View File

@ -13,7 +13,6 @@
#ifndef EXECPARTITION_H
#define EXECPARTITION_H
#include "catalog/partition.h"
#include "nodes/execnodes.h"
#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"

View File

@ -14,7 +14,6 @@
#ifndef EXECUTOR_H
#define EXECUTOR_H
#include "catalog/partition.h"
#include "executor/execdesc.h"
#include "nodes/parsenodes.h"
#include "utils/memutils.h"

View File

@ -26,6 +26,8 @@
#include "nodes/lockoptions.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
#include "partitioning/partdefs.h"
typedef enum OverridingKind
{
@ -803,7 +805,7 @@ typedef struct PartitionSpec
* This represents the portion of the partition key space assigned to a
* particular partition. These are stored on disk in pg_class.relpartbound.
*/
typedef struct PartitionBoundSpec
struct PartitionBoundSpec
{
NodeTag type;
@ -822,7 +824,7 @@ typedef struct PartitionBoundSpec
List *upperdatums; /* List of PartitionRangeDatums */
int location; /* token location, or -1 if unknown */
} PartitionBoundSpec;
};
/*
* PartitionRangeDatum - one of the values in a range partition bound

View File

@ -11,7 +11,11 @@
#ifndef PARTBOUNDS_H
#define PARTBOUNDS_H
#include "catalog/partition.h"
#include "fmgr.h"
#include "nodes/parsenodes.h"
#include "nodes/pg_list.h"
#include "partitioning/partdefs.h"
#include "utils/relcache.h"
/*
@ -101,7 +105,34 @@ typedef struct PartitionRangeBound
} PartitionRangeBound;
extern int get_hash_partition_greatest_modulus(PartitionBoundInfo b);
extern int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
extern uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
Datum *values, bool *isnull);
extern List *get_qual_from_partbound(Relation rel, Relation parent,
PartitionBoundSpec *spec);
extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
bool *parttypbyval, PartitionBoundInfo b1,
PartitionBoundInfo b2);
extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
PartitionKey key);
extern void check_new_partition_bound(char *relname, Relation parent,
PartitionBoundSpec *spec);
extern void check_default_allows_bound(Relation parent, Relation defaultRel,
PartitionBoundSpec *new_spec);
extern PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
List *datums, bool lower);
extern int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
int remainder2);
extern int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
Oid *partcollation, Datum *datums1,
PartitionRangeDatumKind *kind1, bool lower1,
PartitionRangeBound *b2);
extern int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
Oid *partcollation,
Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
Datum *tuple_datums, int n_tuple_datums);
extern int partition_list_bsearch(FmgrInfo *partsupfunc,
Oid *partcollation,
PartitionBoundInfo boundinfo,
Datum value, bool *is_equal);
extern int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
@ -114,11 +145,5 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
int nvalues, Datum *values, bool *is_equal);
extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
int modulus, int remainder);
extern uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
Datum *values, bool *isnull);
extern int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
Oid *partcollation,
Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
Datum *tuple_datums, int n_tuple_datums);
#endif /* PARTBOUNDS_H */

View File

@ -0,0 +1,24 @@
/*-------------------------------------------------------------------------
*
* partdefs.h
* Base definitions for partitioned table handling
*
* Copyright (c) 2007-2018, PostgreSQL Global Development Group
*
* src/include/partitioning/partdefs.h
*
*-------------------------------------------------------------------------
*/
#ifndef PARTDEFS_H
#define PARTDEFS_H
typedef struct PartitionBoundInfoData *PartitionBoundInfo;
typedef struct PartitionKeyData *PartitionKey;
typedef struct PartitionBoundSpec PartitionBoundSpec;
typedef struct PartitionDescData *PartitionDesc;
#endif /* PARTDEFS_H */

View File

@ -14,9 +14,10 @@
#ifndef PARTPRUNE_H
#define PARTPRUNE_H
#include "catalog/partition.h"
#include "nodes/execnodes.h"
#include "nodes/relation.h"
/*
* PartitionPruneContext
*

View File

@ -0,0 +1,96 @@
/*-------------------------------------------------------------------------
*
* partcache.h
*
* Copyright (c) 1996-2018, PostgreSQL Global Development Group
*
* src/include/utils/partcache.h
*
*-------------------------------------------------------------------------
*/
#ifndef PARTCACHE_H
#define PARTCACHE_H
#include "access/attnum.h"
#include "fmgr.h"
#include "nodes/pg_list.h"
#include "nodes/primnodes.h"
#include "partitioning/partdefs.h"
#include "utils/relcache.h"
/*
* Information about the partition key of a relation
*/
typedef struct PartitionKeyData
{
char strategy; /* partitioning strategy */
int16 partnatts; /* number of columns in the partition key */
AttrNumber *partattrs; /* attribute numbers of columns in the
* partition key */
List *partexprs; /* list of expressions in the partitioning
* key, or NIL */
Oid *partopfamily; /* OIDs of operator families */
Oid *partopcintype; /* OIDs of opclass declared input data types */
FmgrInfo *partsupfunc; /* lookup info for support funcs */
/* Partitioning collation per attribute */
Oid *partcollation;
/* Type information per attribute */
Oid *parttypid;
int32 *parttypmod;
int16 *parttyplen;
bool *parttypbyval;
char *parttypalign;
Oid *parttypcoll;
} PartitionKeyData;
extern void RelationBuildPartitionKey(Relation relation);
extern void RelationBuildPartitionDesc(Relation rel);
extern List *RelationGetPartitionQual(Relation rel);
extern Expr *get_partition_qual_relid(Oid relid);
/*
* PartitionKey inquiry functions
*/
static inline int
get_partition_strategy(PartitionKey key)
{
return key->strategy;
}
static inline int
get_partition_natts(PartitionKey key)
{
return key->partnatts;
}
static inline List *
get_partition_exprs(PartitionKey key)
{
return key->partexprs;
}
/*
* PartitionKey inquiry functions - one column
*/
static inline int16
get_partition_col_attnum(PartitionKey key, int col)
{
return key->partattrs[col];
}
static inline Oid
get_partition_col_typid(PartitionKey key, int col)
{
return key->parttypid[col];
}
static inline int32
get_partition_col_typmod(PartitionKey key, int col)
{
return key->parttypmod[col];
}
#endif /* PARTCACHE_H */

View File

@ -46,36 +46,6 @@ typedef struct LockInfoData
typedef LockInfoData *LockInfo;
/*
* Information about the partition key of a relation
*/
typedef struct PartitionKeyData
{
char strategy; /* partitioning strategy */
int16 partnatts; /* number of columns in the partition key */
AttrNumber *partattrs; /* attribute numbers of columns in the
* partition key */
List *partexprs; /* list of expressions in the partitioning
* key, or NIL */
Oid *partopfamily; /* OIDs of operator families */
Oid *partopcintype; /* OIDs of opclass declared input data types */
FmgrInfo *partsupfunc; /* lookup info for support funcs */
/* Partitioning collation per attribute */
Oid *partcollation;
/* Type information per attribute */
Oid *parttypid;
int32 *parttypmod;
int16 *parttyplen;
bool *parttypbyval;
char *parttypalign;
Oid *parttypcoll;
} PartitionKeyData;
typedef struct PartitionKeyData *PartitionKey;
/*
* Here are the contents of a relation cache entry.
*/
@ -618,48 +588,6 @@ typedef struct ViewOptions
*/
#define RelationGetPartitionKey(relation) ((relation)->rd_partkey)
/*
* PartitionKey inquiry functions
*/
static inline int
get_partition_strategy(PartitionKey key)
{
return key->strategy;
}
static inline int
get_partition_natts(PartitionKey key)
{
return key->partnatts;
}
static inline List *
get_partition_exprs(PartitionKey key)
{
return key->partexprs;
}
/*
* PartitionKey inquiry functions - one column
*/
static inline int16
get_partition_col_attnum(PartitionKey key, int col)
{
return key->partattrs[col];
}
static inline Oid
get_partition_col_typid(PartitionKey key, int col)
{
return key->parttypid[col];
}
static inline int32
get_partition_col_typmod(PartitionKey key, int col)
{
return key->parttypmod[col];
}
/*
* RelationGetPartitionDesc
* Returns partition descriptor for a relation.