1
0
mirror of https://github.com/postgres/postgres.git synced 2025-07-05 07:21:24 +03:00

Add hash partitioning.

Hash partitioning is useful when you want to partition a growing data
set evenly.  This can be useful to keep table sizes reasonable, which
makes maintenance operations such as VACUUM faster, or to enable
partition-wise join.

At present, we still depend on constraint exclusion for partitioning
pruning, and the shape of the partition constraints for hash
partitioning is such that that doesn't work.  Work is underway to fix
that, which should both improve performance and make partitioning
pruning work with hash partitioning.

Amul Sul, reviewed and tested by Dilip Kumar, Ashutosh Bapat, Yugo
Nagata, Rajkumar Raghuwanshi, Jesper Pedersen, and by me.  A few
final tweaks also by me.

Discussion: http://postgr.es/m/CAAJ_b96fhpJAP=ALbETmeLk1Uni_GFZD938zgenhF49qgDTjaQ@mail.gmail.com
This commit is contained in:
Robert Haas
2017-11-09 18:07:25 -05:00
parent e7397f015c
commit 1aba8e651a
30 changed files with 1421 additions and 121 deletions

View File

@ -15,6 +15,7 @@
#include "postgres.h"
#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@ -46,6 +47,7 @@
#include "utils/datum.h"
#include "utils/memutils.h"
#include "utils/fmgroids.h"
#include "utils/hashutils.h"
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/rel.h"
@ -61,26 +63,35 @@
* In the case of range partitioning, ndatums will typically be far less than
* 2 * nparts, because a partition's upper bound and the next partition's lower
* bound are the same in most common cases, and we only store one of them (the
* upper bound).
* upper bound). In case of hash partitioning, ndatums will be same as the
* number of partitions.
*
* For range and list partitioned tables, datums is an array of datum-tuples
* with key->partnatts datums each. For hash partitioned tables, it is an array
* of datum-tuples with 2 datums, modulus and remainder, corresponding to a
* given partition.
*
* In the case of list partitioning, the indexes array stores one entry for
* every datum, which is the index of the partition that accepts a given datum.
* In case of range partitioning, it stores one entry per distinct range
* datum, which is the index of the partition for which a given datum
* is an upper bound.
* is an upper bound. In the case of hash partitioning, the number of the
* entries in the indexes array is same as the greatest modulus amongst all
* partitions. For a given partition key datum-tuple, the index of the
* partition which would accept that datum-tuple would be given by the entry
* pointed by remainder produced when hash value of the datum-tuple is divided
* by the greatest modulus.
*/
typedef struct PartitionBoundInfoData
{
char strategy; /* list or range bounds? */
char strategy; /* hash, list or range? */
int ndatums; /* Length of the datums following array */
Datum **datums; /* Array of datum-tuples with key->partnatts
* datums each */
Datum **datums;
PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
* NULL for list partitioned tables */
int *indexes; /* Partition indexes; one entry per member of
* the datums array (plus one if range
* partitioned table) */
* NULL for hash and list partitioned
* tables */
int *indexes; /* Partition indexes */
int null_index; /* Index of the null-accepting partition; -1
* if there isn't one */
int default_index; /* Index of the default partition; -1 if there
@ -95,6 +106,14 @@ typedef struct PartitionBoundInfoData
* is represented with one of the following structs.
*/
/* One bound of a hash partition */
typedef struct PartitionHashBound
{
int modulus;
int remainder;
int index;
} PartitionHashBound;
/* One value coming from some (index'th) list partition */
typedef struct PartitionListValue
{
@ -111,6 +130,7 @@ typedef struct PartitionRangeBound
bool lower; /* this is the lower (vs upper) bound */
} PartitionRangeBound;
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,
@ -126,6 +146,7 @@ static void get_range_key_properties(PartitionKey key, int keynum,
ListCell **partexprs_item,
Expr **keyCol,
Const **lower_val, Const **upper_val);
static List *get_qual_for_hash(Relation parent, PartitionBoundSpec *spec);
static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
bool for_default);
@ -134,6 +155,8 @@ static List *generate_partition_qual(Relation rel);
static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
List *datums, bool lower);
static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
int remainder2);
static int32 partition_rbound_cmp(PartitionKey key,
Datum *datums1, PartitionRangeDatumKind *kind1,
bool lower1, PartitionRangeBound *b2);
@ -149,6 +172,12 @@ static int partition_bound_bsearch(PartitionKey key,
void *probe, bool probe_is_bound, bool *is_equal);
static void get_partition_dispatch_recurse(Relation rel, Relation parent,
List **pds, List **leaf_part_oids);
static int get_partition_bound_num_indexes(PartitionBoundInfo b);
static int get_greatest_modulus(PartitionBoundInfo b);
static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
/* SQL-callable function for use in hash partition CHECK constraints */
PG_FUNCTION_INFO_V1(satisfies_hash_partition);
/*
* RelationBuildPartitionDesc
@ -174,6 +203,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
int default_index = -1;
/* Hash partitioning specific */
PartitionHashBound **hbounds = NULL;
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@ -255,7 +287,35 @@ RelationBuildPartitionDesc(Relation rel)
oids[i++] = lfirst_oid(cell);
/* Convert from node to the internal representation */
if (key->strategy == PARTITION_STRATEGY_LIST)
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;
@ -484,6 +544,42 @@ RelationBuildPartitionDesc(Relation rel)
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));
@ -617,8 +713,7 @@ RelationBuildPartitionDesc(Relation rel)
* 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 list values or the
* range bounds.
* defined by canonicalized representation of the partition bounds.
*/
for (i = 0; i < nparts; i++)
result->oids[mapping[i]] = oids[i];
@ -655,49 +750,97 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
if (b1->default_index != b2->default_index)
return false;
for (i = 0; i < b1->ndatums; i++)
if (b1->strategy == PARTITION_STRATEGY_HASH)
{
int j;
int greatest_modulus;
for (j = 0; j < partnatts; j++)
/*
* If two hash partitioned tables have different greatest moduli,
* their partition schemes don't match. For hash partitioned table,
* the greatest modulus is given by the last datum and number of
* partitions is given by ndatums.
*/
if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
return false;
/*
* We arrange the partitions in the ascending order of their modulus
* and remainders. Also every modulus is factor of next larger
* modulus. Therefore we can safely store index of a given partition
* in indexes array at remainder of that partition. Also entries at
* (remainder + N * modulus) positions in indexes array are all same
* for (modulus, remainder) specification for any partition. Thus
* datums array from both the given bounds are same, if and only if
* their indexes array will be same. So, it suffices to compare
* indexes array.
*/
greatest_modulus = get_greatest_modulus(b1);
for (i = 0; i < greatest_modulus; i++)
if (b1->indexes[i] != b2->indexes[i])
return false;
#ifdef USE_ASSERT_CHECKING
/*
* Nonetheless make sure that the bounds are indeed same when the
* indexes match. Hash partition bound stores modulus and remainder
* at b1->datums[i][0] and b1->datums[i][1] position respectively.
*/
for (i = 0; i < b1->ndatums; i++)
Assert((b1->datums[i][0] == b2->datums[i][0] &&
b1->datums[i][1] == b2->datums[i][1]));
#endif
}
else
{
for (i = 0; i < b1->ndatums; i++)
{
/* For range partitions, the bounds might not be finite. */
if (b1->kind != NULL)
{
/* The different kinds of bound all differ from each other */
if (b1->kind[i][j] != b2->kind[i][j])
return false;
int j;
/* Non-finite bounds are equal without further examination. */
if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
continue;
for (j = 0; j < partnatts; j++)
{
/* For range partitions, the bounds might not be finite. */
if (b1->kind != NULL)
{
/* The different kinds of bound all differ from each other */
if (b1->kind[i][j] != b2->kind[i][j])
return false;
/*
* Non-finite bounds are equal without further
* examination.
*/
if (b1->kind[i][j] != PARTITION_RANGE_DATUM_VALUE)
continue;
}
/*
* Compare the actual values. Note that it would be both
* incorrect and unsafe to invoke the comparison operator
* derived from the partitioning specification here. It would
* be incorrect because we want the relcache entry to be
* updated for ANY change to the partition bounds, not just
* those that the partitioning operator thinks are
* significant. It would be unsafe because we might reach
* this code in the context of an aborted transaction, and an
* arbitrary partitioning operator might not be safe in that
* context. datumIsEqual() should be simple enough to be
* safe.
*/
if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
parttypbyval[j], parttyplen[j]))
return false;
}
/*
* Compare the actual values. Note that it would be both incorrect
* and unsafe to invoke the comparison operator derived from the
* partitioning specification here. It would be incorrect because
* we want the relcache entry to be updated for ANY change to the
* partition bounds, not just those that the partitioning operator
* thinks are significant. It would be unsafe because we might
* reach this code in the context of an aborted transaction, and
* an arbitrary partitioning operator might not be safe in that
* context. datumIsEqual() should be simple enough to be safe.
*/
if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
parttypbyval[j], parttyplen[j]))
if (b1->indexes[i] != b2->indexes[i])
return false;
}
if (b1->indexes[i] != b2->indexes[i])
/* There are ndatums+1 indexes in case of range partitions */
if (b1->strategy == PARTITION_STRATEGY_RANGE &&
b1->indexes[i] != b2->indexes[i])
return false;
}
/* There are ndatums+1 indexes in case of range partitions */
if (b1->strategy == PARTITION_STRATEGY_RANGE &&
b1->indexes[i] != b2->indexes[i])
return false;
return true;
}
@ -709,11 +852,11 @@ extern PartitionBoundInfo
partition_bounds_copy(PartitionBoundInfo src,
PartitionKey key)
{
PartitionBoundInfo dest;
int i;
int ndatums;
int partnatts;
int num_indexes;
PartitionBoundInfo dest;
int i;
int ndatums;
int partnatts;
int num_indexes;
dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
@ -721,8 +864,7 @@ partition_bounds_copy(PartitionBoundInfo src,
ndatums = dest->ndatums = src->ndatums;
partnatts = key->partnatts;
/* Range partitioned table has an extra index. */
num_indexes = key->strategy == PARTITION_STRATEGY_RANGE ? ndatums + 1 : ndatums;
num_indexes = get_partition_bound_num_indexes(src);
/* List partitioned tables have only a single partition key. */
Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
@ -732,11 +874,11 @@ partition_bounds_copy(PartitionBoundInfo src,
if (src->kind != NULL)
{
dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
sizeof(PartitionRangeDatumKind *));
sizeof(PartitionRangeDatumKind *));
for (i = 0; i < ndatums; i++)
{
dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
sizeof(PartitionRangeDatumKind));
sizeof(PartitionRangeDatumKind));
memcpy(dest->kind[i], src->kind[i],
sizeof(PartitionRangeDatumKind) * key->partnatts);
@ -747,16 +889,37 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
int j;
for (j = 0; j < partnatts; j++)
/*
* For a corresponding to hash partition, datums array will have two
* elements - modulus and remainder.
*/
bool hash_part = (key->strategy == PARTITION_STRATEGY_HASH);
int natts = hash_part ? 2 : partnatts;
dest->datums[i] = (Datum *) palloc(sizeof(Datum) * natts);
for (j = 0; j < natts; j++)
{
bool byval;
int typlen;
if (hash_part)
{
typlen = sizeof(int32); /* Always int4 */
byval = true; /* int4 is pass-by-value */
}
else
{
byval = key->parttypbyval[j];
typlen = key->parttyplen[j];
}
if (dest->kind == NULL ||
dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
dest->datums[i][j] = datumCopy(src->datums[i][j],
key->parttypbyval[j],
key->parttyplen[j]);
byval, typlen);
}
}
@ -801,6 +964,89 @@ check_new_partition_bound(char *relname, Relation parent,
switch (key->strategy)
{
case PARTITION_STRATEGY_HASH:
{
Assert(spec->strategy == PARTITION_STRATEGY_HASH);
Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
if (partdesc->nparts > 0)
{
PartitionBoundInfo boundinfo = partdesc->boundinfo;
Datum **datums = boundinfo->datums;
int ndatums = boundinfo->ndatums;
int greatest_modulus;
int remainder;
int offset;
bool equal,
valid_modulus = true;
int prev_modulus, /* Previous largest modulus */
next_modulus; /* Next largest modulus */
/*
* Check rule that every modulus must be a factor of the
* next larger modulus. For example, if you have a bunch
* of partitions that all have modulus 5, you can add a
* new partition with modulus 10 or a new partition with
* modulus 15, but you cannot add both a partition with
* modulus 10 and a partition with modulus 15, because 10
* is not a factor of 15.
*
* Get greatest bound in array boundinfo->datums which is
* less than or equal to spec->modulus and
* spec->remainder.
*/
offset = partition_bound_bsearch(key, boundinfo, spec,
true, &equal);
if (offset < 0)
{
next_modulus = DatumGetInt32(datums[0][0]);
valid_modulus = (next_modulus % spec->modulus) == 0;
}
else
{
prev_modulus = DatumGetInt32(datums[offset][0]);
valid_modulus = (spec->modulus % prev_modulus) == 0;
if (valid_modulus && (offset + 1) < ndatums)
{
next_modulus = DatumGetInt32(datums[offset + 1][0]);
valid_modulus = (next_modulus % spec->modulus) == 0;
}
}
if (!valid_modulus)
ereport(ERROR,
(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
errmsg("every hash partition modulus must be a factor of the next larger modulus")));
greatest_modulus = get_greatest_modulus(boundinfo);
remainder = spec->remainder;
/*
* Normally, the lowest remainder that could conflict with
* the new partition is equal to the remainder specified
* for the new partition, but when the new partition has a
* modulus higher than any used so far, we need to adjust.
*/
if (remainder >= greatest_modulus)
remainder = remainder % greatest_modulus;
/* Check every potentially-conflicting remainder. */
do
{
if (boundinfo->indexes[remainder] != -1)
{
overlap = true;
with = boundinfo->indexes[remainder];
break;
}
remainder += spec->modulus;
} while (remainder < greatest_modulus);
}
break;
}
case PARTITION_STRATEGY_LIST:
{
Assert(spec->strategy == PARTITION_STRATEGY_LIST);
@ -1171,6 +1417,11 @@ get_qual_from_partbound(Relation rel, Relation parent,
switch (key->strategy)
{
case PARTITION_STRATEGY_HASH:
Assert(spec->strategy == PARTITION_STRATEGY_HASH);
my_qual = get_qual_for_hash(parent, spec);
break;
case PARTITION_STRATEGY_LIST:
Assert(spec->strategy == PARTITION_STRATEGY_LIST);
my_qual = get_qual_for_list(parent, spec);
@ -1541,6 +1792,92 @@ make_partition_op_expr(PartitionKey key, int keynum,
return result;
}
/*
* get_qual_for_hash
*
* Given a list of partition columns, modulus and remainder corresponding to a
* partition, this function returns CHECK constraint expression Node for that
* partition.
*
* The partition constraint for a hash partition is always a call to the
* built-in function satisfies_hash_partition(). The first two arguments are
* the modulus and remainder for the partition; the remaining arguments are the
* values to be hashed.
*/
static List *
get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
{
PartitionKey key = RelationGetPartitionKey(parent);
FuncExpr *fexpr;
Node *relidConst;
Node *modulusConst;
Node *remainderConst;
List *args;
ListCell *partexprs_item;
int i;
/* Fixed arguments. */
relidConst = (Node *) makeConst(OIDOID,
-1,
InvalidOid,
sizeof(Oid),
ObjectIdGetDatum(RelationGetRelid(parent)),
false,
true);
modulusConst = (Node *) makeConst(INT4OID,
-1,
InvalidOid,
sizeof(int32),
Int32GetDatum(spec->modulus),
false,
true);
remainderConst = (Node *) makeConst(INT4OID,
-1,
InvalidOid,
sizeof(int32),
Int32GetDatum(spec->remainder),
false,
true);
args = list_make3(relidConst, modulusConst, remainderConst);
partexprs_item = list_head(key->partexprs);
/* Add an argument for each key column. */
for (i = 0; i < key->partnatts; i++)
{
Node *keyCol;
/* Left operand */
if (key->partattrs[i] != 0)
{
keyCol = (Node *) makeVar(1,
key->partattrs[i],
key->parttypid[i],
key->parttypmod[i],
key->parttypcoll[i],
0);
}
else
{
keyCol = (Node *) copyObject(lfirst(partexprs_item));
partexprs_item = lnext(partexprs_item);
}
args = lappend(args, keyCol);
}
fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
BOOLOID,
args,
InvalidOid,
InvalidOid,
COERCE_EXPLICIT_CALL);
return list_make1(fexpr);
}
/*
* get_qual_for_list
*
@ -2412,6 +2749,17 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* Route as appropriate based on partitioning strategy. */
switch (key->strategy)
{
case PARTITION_STRATEGY_HASH:
{
PartitionBoundInfo boundinfo = partdesc->boundinfo;
int greatest_modulus = get_greatest_modulus(boundinfo);
uint64 rowHash = compute_hash_value(key, values,
isnull);
cur_index = boundinfo->indexes[rowHash % greatest_modulus];
}
break;
case PARTITION_STRATEGY_LIST:
if (isnull[0])
@ -2524,6 +2872,38 @@ error_exit:
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);
}
/*
* partition_hbound_cmp
*
* Compares modulus first, then remainder if modulus are equal.
*/
static int32
partition_hbound_cmp(int modulus1, int remainder1, int modulus2, int remainder2)
{
if (modulus1 < modulus2)
return -1;
if (modulus1 > modulus2)
return 1;
if (modulus1 == modulus2 && remainder1 != remainder2)
return (remainder1 > remainder2) ? 1 : -1;
return 0;
}
/*
* qsort_partition_list_value_cmp
*
@ -2710,6 +3090,15 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
switch (key->strategy)
{
case PARTITION_STRATEGY_HASH:
{
PartitionBoundSpec *spec = (PartitionBoundSpec *) probe;
cmpval = partition_hbound_cmp(DatumGetInt32(bound_datums[0]),
DatumGetInt32(bound_datums[1]),
spec->modulus, spec->remainder);
break;
}
case PARTITION_STRATEGY_LIST:
cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
@ -2894,3 +3283,182 @@ get_proposed_default_constraint(List *new_part_constraints)
return list_make1(defPartConstraint);
}
/*
* get_partition_bound_num_indexes
*
* Returns the number of the entries in the partition bound indexes array.
*/
static int
get_partition_bound_num_indexes(PartitionBoundInfo bound)
{
int num_indexes;
Assert(bound);
switch (bound->strategy)
{
case PARTITION_STRATEGY_HASH:
/*
* The number of the entries in the indexes array is same as the
* greatest modulus.
*/
num_indexes = get_greatest_modulus(bound);
break;
case PARTITION_STRATEGY_LIST:
num_indexes = bound->ndatums;
break;
case PARTITION_STRATEGY_RANGE:
/* Range partitioned table has an extra index. */
num_indexes = bound->ndatums + 1;
break;
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) bound->strategy);
}
return num_indexes;
}
/*
* get_greatest_modulus
*
* Returns the greatest modulus of the hash partition bound. The greatest
* modulus will be at the end of the datums array because hash partitions are
* arranged in the ascending order of their modulus and remainders.
*/
static int
get_greatest_modulus(PartitionBoundInfo bound)
{
Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
Assert(bound->datums && bound->ndatums > 0);
Assert(DatumGetInt32(bound->datums[bound->ndatums - 1][0]) > 0);
return DatumGetInt32(bound->datums[bound->ndatums - 1][0]);
}
/*
* compute_hash_value
*
* Compute the hash value for given not null partition key values.
*/
static uint64
compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
{
int i;
int nkeys = key->partnatts;
uint64 rowHash = 0;
Datum seed = UInt64GetDatum(HASH_PARTITION_SEED);
for (i = 0; i < nkeys; i++)
{
if (!isnull[i])
{
Datum hash;
Assert(OidIsValid(key->partsupfunc[i].fn_oid));
/*
* Compute hash for each datum value by calling respective
* datatype-specific hash functions of each partition key
* attribute.
*/
hash = FunctionCall2(&key->partsupfunc[i], values[i], seed);
/* Form a single 64-bit hash value */
rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
}
}
return rowHash;
}
/*
* satisfies_hash_partition
*
* This is a SQL-callable function for use in hash partition constraints takes
* an already computed hash values of each partition key attribute, and combine
* them into a single hash value by calling hash_combine64.
*
* Returns true if remainder produced when this computed single hash value is
* divided by the given modulus is equal to given remainder, otherwise false.
*
* See get_qual_for_hash() for usage.
*/
Datum
satisfies_hash_partition(PG_FUNCTION_ARGS)
{
typedef struct ColumnsHashData
{
Oid relid;
int16 nkeys;
FmgrInfo partsupfunc[PARTITION_MAX_KEYS];
} ColumnsHashData;
Oid parentId = PG_GETARG_OID(0);
int modulus = PG_GETARG_INT32(1);
int remainder = PG_GETARG_INT32(2);
short nkeys = PG_NARGS() - 3;
int i;
Datum seed = UInt64GetDatum(HASH_PARTITION_SEED);
ColumnsHashData *my_extra;
uint64 rowHash = 0;
/*
* Cache hash function information.
*/
my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
if (my_extra == NULL || my_extra->nkeys != nkeys ||
my_extra->relid != parentId)
{
Relation parent;
PartitionKey key;
int j;
fcinfo->flinfo->fn_extra =
MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
offsetof(ColumnsHashData, partsupfunc) +
sizeof(FmgrInfo) * nkeys);
my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
my_extra->nkeys = nkeys;
my_extra->relid = parentId;
/* Open parent relation and fetch partition keyinfo */
parent = heap_open(parentId, AccessShareLock);
key = RelationGetPartitionKey(parent);
Assert(key->partnatts == nkeys);
for (j = 0; j < nkeys; ++j)
fmgr_info_copy(&my_extra->partsupfunc[j],
key->partsupfunc,
fcinfo->flinfo->fn_mcxt);
/* Hold lock until commit */
heap_close(parent, NoLock);
}
for (i = 0; i < nkeys; i++)
{
/* keys start from fourth argument of function. */
int argno = i + 3;
if (!PG_ARGISNULL(argno))
{
Datum hash;
Assert(OidIsValid(my_extra->partsupfunc[i].fn_oid));
hash = FunctionCall2(&my_extra->partsupfunc[i],
PG_GETARG_DATUM(argno),
seed);
/* Form a single 64-bit hash value */
rowHash = hash_combine64(rowHash, DatumGetUInt64(hash));
}
}
PG_RETURN_BOOL(rowHash % modulus == remainder);
}

View File

@ -471,7 +471,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid,
static bool is_partition_attr(Relation rel, AttrNumber attnum, bool *used_in_expr);
static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy);
static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
List **partexprs, Oid *partopclass, Oid *partcollation);
List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
static void CreateInheritance(Relation child_rel, Relation parent_rel);
static void RemoveInheritance(Relation child_rel, Relation parent_rel);
static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
@ -894,7 +894,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
partcollation);
partcollation, strategy);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
partopclass, partcollation);
@ -13337,7 +13337,9 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
newspec->location = partspec->location;
/* Parse partitioning strategy name */
if (pg_strcasecmp(partspec->strategy, "list") == 0)
if (pg_strcasecmp(partspec->strategy, "hash") == 0)
*strategy = PARTITION_STRATEGY_HASH;
else if (pg_strcasecmp(partspec->strategy, "list") == 0)
*strategy = PARTITION_STRATEGY_LIST;
else if (pg_strcasecmp(partspec->strategy, "range") == 0)
*strategy = PARTITION_STRATEGY_RANGE;
@ -13407,10 +13409,12 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*/
static void
ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
List **partexprs, Oid *partopclass, Oid *partcollation)
List **partexprs, Oid *partopclass, Oid *partcollation,
char strategy)
{
int attn;
ListCell *lc;
Oid am_oid;
attn = 0;
foreach(lc, partParams)
@ -13570,25 +13574,41 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
partcollation[attn] = attcollation;
/*
* Identify a btree opclass to use. Currently, we use only btree
* operators, which seems enough for list and range partitioning.
* Identify the appropriate operator class. For list and range
* partitioning, we use a btree operator class; hash partitioning uses
* a hash operator class.
*/
if (strategy == PARTITION_STRATEGY_HASH)
am_oid = HASH_AM_OID;
else
am_oid = BTREE_AM_OID;
if (!pelem->opclass)
{
partopclass[attn] = GetDefaultOpClass(atttype, BTREE_AM_OID);
partopclass[attn] = GetDefaultOpClass(atttype, am_oid);
if (!OidIsValid(partopclass[attn]))
ereport(ERROR,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("data type %s has no default btree operator class",
format_type_be(atttype)),
errhint("You must specify a btree operator class or define a default btree operator class for the data type.")));
{
if (strategy == PARTITION_STRATEGY_HASH)
ereport(ERROR,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("data type %s has no default hash operator class",
format_type_be(atttype)),
errhint("You must specify a hash operator class or define a default hash operator class for the data type.")));
else
ereport(ERROR,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("data type %s has no default btree operator class",
format_type_be(atttype)),
errhint("You must specify a btree operator class or define a default btree operator class for the data type.")));
}
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
"btree",
BTREE_AM_OID);
am_oid == HASH_AM_OID ? "hash" : "btree",
am_oid);
attn++;
}

View File

@ -4461,6 +4461,8 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
COPY_SCALAR_FIELD(strategy);
COPY_SCALAR_FIELD(is_default);
COPY_SCALAR_FIELD(modulus);
COPY_SCALAR_FIELD(remainder);
COPY_NODE_FIELD(listdatums);
COPY_NODE_FIELD(lowerdatums);
COPY_NODE_FIELD(upperdatums);

View File

@ -2848,6 +2848,8 @@ _equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *
{
COMPARE_SCALAR_FIELD(strategy);
COMPARE_SCALAR_FIELD(is_default);
COMPARE_SCALAR_FIELD(modulus);
COMPARE_SCALAR_FIELD(remainder);
COMPARE_NODE_FIELD(listdatums);
COMPARE_NODE_FIELD(lowerdatums);
COMPARE_NODE_FIELD(upperdatums);

View File

@ -3578,6 +3578,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_CHAR_FIELD(strategy);
WRITE_BOOL_FIELD(is_default);
WRITE_INT_FIELD(modulus);
WRITE_INT_FIELD(remainder);
WRITE_NODE_FIELD(listdatums);
WRITE_NODE_FIELD(lowerdatums);
WRITE_NODE_FIELD(upperdatums);

View File

@ -2397,6 +2397,8 @@ _readPartitionBoundSpec(void)
READ_CHAR_FIELD(strategy);
READ_BOOL_FIELD(is_default);
READ_INT_FIELD(modulus);
READ_INT_FIELD(remainder);
READ_NODE_FIELD(listdatums);
READ_NODE_FIELD(lowerdatums);
READ_NODE_FIELD(upperdatums);

View File

@ -1463,7 +1463,7 @@ have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2, JoinType jointype,
continue;
/* Skip clauses which are not equality conditions. */
if (!rinfo->mergeopfamilies)
if (!rinfo->mergeopfamilies && !OidIsValid(rinfo->hashjoinoperator))
continue;
opexpr = (OpExpr *) rinfo->clause;
@ -1515,8 +1515,14 @@ have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2, JoinType jointype,
* The clause allows partition-wise join if only it uses the same
* operator family as that specified by the partition key.
*/
if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
if (rel1->part_scheme->strategy == PARTITION_STRATEGY_HASH)
{
if (!op_in_opfamily(rinfo->hashjoinoperator,
part_scheme->partopfamily[ipk1]))
continue;
}
else if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
continue;
/* Mark the partition key as having an equi-join clause. */

View File

@ -579,7 +579,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <list> part_params
%type <partboundspec> PartitionBoundSpec
%type <node> partbound_datum PartitionRangeDatum
%type <list> partbound_datum_list range_datum_list
%type <list> hash_partbound partbound_datum_list range_datum_list
%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@ -2638,8 +2639,61 @@ alter_identity_column_option:
;
PartitionBoundSpec:
/* a HASH partition*/
FOR VALUES WITH '(' hash_partbound ')'
{
ListCell *lc;
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
n->strategy = PARTITION_STRATEGY_HASH;
n->modulus = n->remainder = -1;
foreach (lc, $5)
{
DefElem *opt = lfirst_node(DefElem, lc);
if (strcmp(opt->defname, "modulus") == 0)
{
if (n->modulus != -1)
ereport(ERROR,
(errcode(ERRCODE_DUPLICATE_OBJECT),
errmsg("modulus for hash partition provided more than once"),
parser_errposition(opt->location)));
n->modulus = defGetInt32(opt);
}
else if (strcmp(opt->defname, "remainder") == 0)
{
if (n->remainder != -1)
ereport(ERROR,
(errcode(ERRCODE_DUPLICATE_OBJECT),
errmsg("remainder for hash partition provided more than once"),
parser_errposition(opt->location)));
n->remainder = defGetInt32(opt);
}
else
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("unrecognized hash partition bound specification \"%s\"",
opt->defname),
parser_errposition(opt->location)));
}
if (n->modulus == -1)
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("modulus for hash partition must be specified")));
if (n->remainder == -1)
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("remainder for hash partition must be specified")));
n->location = @3;
$$ = n;
}
/* a LIST partition */
FOR VALUES IN_P '(' partbound_datum_list ')'
| FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@ -2677,6 +2731,24 @@ PartitionBoundSpec:
}
;
hash_partbound_elem:
NonReservedWord Iconst
{
$$ = makeDefElem($1, (Node *)makeInteger($2), @1);
}
;
hash_partbound:
hash_partbound_elem
{
$$ = list_make1($1);
}
| hash_partbound ',' hash_partbound_elem
{
$$ = lappend($1, $3);
}
;
partbound_datum:
Sconst { $$ = makeStringConst($1, @1); }
| NumericOnly { $$ = makeAConst($1, @1); }

View File

@ -3310,6 +3310,11 @@ transformPartitionBound(ParseState *pstate, Relation parent,
if (spec->is_default)
{
if (strategy == PARTITION_STRATEGY_HASH)
ereport(ERROR,
(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("a hash-partitioned table may not have a default partition")));
/*
* In case of the default partition, parser had no way to identify the
* partition strategy. Assign the parent's strategy to the default
@ -3320,7 +3325,27 @@ transformPartitionBound(ParseState *pstate, Relation parent,
return result_spec;
}
if (strategy == PARTITION_STRATEGY_LIST)
if (strategy == PARTITION_STRATEGY_HASH)
{
if (spec->strategy != PARTITION_STRATEGY_HASH)
ereport(ERROR,
(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("invalid bound specification for a hash partition"),
parser_errposition(pstate, exprLocation((Node *) spec))));
if (spec->modulus <= 0)
ereport(ERROR,
(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("modulus for hash partition must be a positive integer")));
Assert(spec->remainder >= 0);
if (spec->remainder >= spec->modulus)
ereport(ERROR,
(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("remainder for hash partition must be less than modulus")));
}
else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
@ -3485,7 +3510,7 @@ transformPartitionBound(ParseState *pstate, Relation parent,
static void
validateInfiniteBounds(ParseState *pstate, List *blist)
{
ListCell *lc;
ListCell *lc;
PartitionRangeDatumKind kind = PARTITION_RANGE_DATUM_VALUE;
foreach(lc, blist)

View File

@ -1551,7 +1551,7 @@ pg_get_statisticsobj_worker(Oid statextid, bool missing_ok)
*
* Returns the partition key specification, ie, the following:
*
* PARTITION BY { RANGE | LIST } (column opt_collation opt_opclass [, ...])
* PARTITION BY { RANGE | LIST | HASH } (column opt_collation opt_opclass [, ...])
*/
Datum
pg_get_partkeydef(PG_FUNCTION_ARGS)
@ -1655,6 +1655,10 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags,
switch (form->partstrat)
{
case PARTITION_STRATEGY_HASH:
if (!attrsOnly)
appendStringInfo(&buf, "HASH");
break;
case PARTITION_STRATEGY_LIST:
if (!attrsOnly)
appendStringInfoString(&buf, "LIST");
@ -8711,6 +8715,15 @@ get_rule_expr(Node *node, deparse_context *context,
switch (spec->strategy)
{
case PARTITION_STRATEGY_HASH:
Assert(spec->modulus > 0 && spec->remainder >= 0);
Assert(spec->modulus > spec->remainder);
appendStringInfoString(buf, "FOR VALUES");
appendStringInfo(buf, " WITH (modulus %d, remainder %d)",
spec->modulus, spec->remainder);
break;
case PARTITION_STRATEGY_LIST:
Assert(spec->listdatums != NIL);

View File

@ -30,6 +30,7 @@
#include <fcntl.h>
#include <unistd.h>
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/multixact.h"
#include "access/nbtree.h"
@ -833,6 +834,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@ -912,6 +914,10 @@ RelationBuildPartitionKey(Relation relation)
key->parttypalign = (char *) palloc0(key->partnatts * sizeof(char));
key->parttypcoll = (Oid *) palloc0(key->partnatts * sizeof(Oid));
/* For the hash partitioning, an extended hash function will be used. */
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);
@ -932,18 +938,20 @@ RelationBuildPartitionKey(Relation relation)
key->partopfamily[i] = opclassform->opcfamily;
key->partopcintype[i] = opclassform->opcintype;
/*
* A btree support function covers the cases of list and range methods
* currently supported.
*/
/* Get a support function for the specified opfamily and datatypes */
funcid = get_opfamily_proc(opclassform->opcfamily,
opclassform->opcintype,
opclassform->opcintype,
BTORDER_PROC);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
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 data type \"%s\"",
NameStr(opclassform->opcname),
(key->strategy == PARTITION_STRATEGY_HASH) ?
"hash" : "btree",
procnum,
format_type_be(opclassform->opcintype))));
fmgr_info(funcid, &key->partsupfunc[i]);