Re: [POC] hash partitioning
Hello
Looking at your hash partitioning syntax, I implemented a hash partition in a more concise way, with no need to determine the number of sub-tables, and dynamically add partitions.
Description
The hash partition's implement is on the basis of the original range / list partition,and using similar syntax.
To create a partitioned table ,use:
CREATE TABLE h (id int) PARTITION BY HASH(id);
The partitioning key supports only one value, and I think the partition key can support multiple values,
which may be difficult to implement when querying, but it is not impossible.
A partition table can be create as bellow:
CREATE TABLE h1 PARTITION OF h;
CREATE TABLE h2 PARTITION OF h;
CREATE TABLE h3 PARTITION OF h;
FOR VALUES clause cannot be used, and the partition bound is calclulated automatically as partition index of single integer value.
An inserted record is stored in a partition whose index equals
DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, values[0])) % nparts/* Number of partitions */
;
In the above example, this is DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, id)) % 3;
postgres=# insert into h select generate_series(1,20);
INSERT 0 20
postgres=# select tableoid::regclass,* from h;
tableoid | id
----------+----
h1 | 3
h1 | 5
h1 | 17
h1 | 19
h2 | 2
h2 | 6
h2 | 7
h2 | 11
h2 | 12
h2 | 14
h2 | 15
h2 | 18
h2 | 20
h3 | 1
h3 | 4
h3 | 8
h3 | 9
h3 | 10
h3 | 13
h3 | 16
(20 rows)
The number of partitions here can be dynamically added, and if a new partition is created, the number of partitions changes, the calculated target partitions will change, and the same data is not reasonable in different partitions,So you need to re-calculate the existing data and insert the target partition when you create a new partition.
postgres=# create table h4 partition of h;
CREATE TABLE
postgres=# select tableoid::regclass,* from h;
tableoid | id
----------+----
h1 | 5
h1 | 17
h1 | 19
h1 | 6
h1 | 12
h1 | 8
h1 | 13
h2 | 11
h2 | 14
h3 | 1
h3 | 9
h3 | 2
h3 | 15
h4 | 3
h4 | 7
h4 | 18
h4 | 20
h4 | 4
h4 | 10
h4 | 16
(20 rows)
When querying the data, the hash partition uses the same algorithm as the insertion, and filters out the table that does not need to be scanned.
postgres=# explain analyze select * from h where id = 1;
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..41.88 rows=13 width=4) (actual time=0.020..0.023 rows=1 loops=1)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4) (actual time=0.013..0.016 rows=1 loops=1)
Filter: (id = 1)
Rows Removed by Filter: 3
Planning time: 0.346 ms
Execution time: 0.061 ms
(6 rows)
postgres=# explain analyze select * from h where id in (1,5);;
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..83.75 rows=52 width=4) (actual time=0.016..0.028 rows=2 loops=1)
-> Seq Scan on h1 (cost=0.00..41.88 rows=26 width=4) (actual time=0.015..0.018 rows=1 loops=1)
Filter: (id = ANY ('{1,5}'::integer[]))
Rows Removed by Filter: 6
-> Seq Scan on h3 (cost=0.00..41.88 rows=26 width=4) (actual time=0.005..0.007 rows=1 loops=1)
Filter: (id = ANY ('{1,5}'::integer[]))
Rows Removed by Filter: 3
Planning time: 0.720 ms
Execution time: 0.074 ms
(9 rows)
postgres=# explain analyze select * from h where id = 1 or id = 5;;
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..96.50 rows=50 width=4) (actual time=0.017..0.078 rows=2 loops=1)
-> Seq Scan on h1 (cost=0.00..48.25 rows=25 width=4) (actual time=0.015..0.019 rows=1 loops=1)
Filter: ((id = 1) OR (id = 5))
Rows Removed by Filter: 6
-> Seq Scan on h3 (cost=0.00..48.25 rows=25 width=4) (actual time=0.005..0.010 rows=1 loops=1)
Filter: ((id = 1) OR (id = 5))
Rows Removed by Filter: 3
Planning time: 0.396 ms
Execution time: 0.139 ms
(9 rows)
Can not detach / attach / drop partition table.
Best regards,
young
yonj1e.github.io
yangjie@highgo.com
Hi young,
On Mon, 28 Aug 2017 15:33:46 +0800
"yangjie@highgo.com" <yangjie@highgo.com> wrote:
Hello
Looking at your hash partitioning syntax, I implemented a hash partition in a more concise way, with no need to determine the number of sub-tables, and dynamically add partitions.
I think it is great work, but the current consensus about hash-partitioning supports
Amul's patch[1]/messages/by-id/CAAJ_b965A2oog=6eFUhELexL3RmgFssB3G7LwkVA1bw0WUJJoA@mail.gmail.com, in which the syntax is different from the my original proposal.
So, you will have to read Amul's patch and make a discussion if you still want to
propose your implementation.
Regards,
[1]: /messages/by-id/CAAJ_b965A2oog=6eFUhELexL3RmgFssB3G7LwkVA1bw0WUJJoA@mail.gmail.com
Description
The hash partition's implement is on the basis of the original range / list partition,and using similar syntax.
To create a partitioned table ,use:
CREATE TABLE h (id int) PARTITION BY HASH(id);
The partitioning key supports only one value, and I think the partition key can support multiple values,
which may be difficult to implement when querying, but it is not impossible.A partition table can be create as bellow:
CREATE TABLE h1 PARTITION OF h;
CREATE TABLE h2 PARTITION OF h;
CREATE TABLE h3 PARTITION OF h;FOR VALUES clause cannot be used, and the partition bound is calclulated automatically as partition index of single integer value.
An inserted record is stored in a partition whose index equals
DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, values[0])) % nparts/* Number of partitions */
;
In the above example, this is DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, id)) % 3;postgres=# insert into h select generate_series(1,20);
INSERT 0 20
postgres=# select tableoid::regclass,* from h;
tableoid | id
----------+----
h1 | 3
h1 | 5
h1 | 17
h1 | 19
h2 | 2
h2 | 6
h2 | 7
h2 | 11
h2 | 12
h2 | 14
h2 | 15
h2 | 18
h2 | 20
h3 | 1
h3 | 4
h3 | 8
h3 | 9
h3 | 10
h3 | 13
h3 | 16
(20 rows)The number of partitions here can be dynamically added, and if a new partition is created, the number of partitions changes, the calculated target partitions will change, and the same data is not reasonable in different partitions,So you need to re-calculate the existing data and insert the target partition when you create a new partition.
postgres=# create table h4 partition of h;
CREATE TABLE
postgres=# select tableoid::regclass,* from h;
tableoid | id
----------+----
h1 | 5
h1 | 17
h1 | 19
h1 | 6
h1 | 12
h1 | 8
h1 | 13
h2 | 11
h2 | 14
h3 | 1
h3 | 9
h3 | 2
h3 | 15
h4 | 3
h4 | 7
h4 | 18
h4 | 20
h4 | 4
h4 | 10
h4 | 16
(20 rows)When querying the data, the hash partition uses the same algorithm as the insertion, and filters out the table that does not need to be scanned.
postgres=# explain analyze select * from h where id = 1;
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..41.88 rows=13 width=4) (actual time=0.020..0.023 rows=1 loops=1)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4) (actual time=0.013..0.016 rows=1 loops=1)
Filter: (id = 1)
Rows Removed by Filter: 3
Planning time: 0.346 ms
Execution time: 0.061 ms
(6 rows)postgres=# explain analyze select * from h where id in (1,5);;
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..83.75 rows=52 width=4) (actual time=0.016..0.028 rows=2 loops=1)
-> Seq Scan on h1 (cost=0.00..41.88 rows=26 width=4) (actual time=0.015..0.018 rows=1 loops=1)
Filter: (id = ANY ('{1,5}'::integer[]))
Rows Removed by Filter: 6
-> Seq Scan on h3 (cost=0.00..41.88 rows=26 width=4) (actual time=0.005..0.007 rows=1 loops=1)
Filter: (id = ANY ('{1,5}'::integer[]))
Rows Removed by Filter: 3
Planning time: 0.720 ms
Execution time: 0.074 ms
(9 rows)postgres=# explain analyze select * from h where id = 1 or id = 5;;
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..96.50 rows=50 width=4) (actual time=0.017..0.078 rows=2 loops=1)
-> Seq Scan on h1 (cost=0.00..48.25 rows=25 width=4) (actual time=0.015..0.019 rows=1 loops=1)
Filter: ((id = 1) OR (id = 5))
Rows Removed by Filter: 6
-> Seq Scan on h3 (cost=0.00..48.25 rows=25 width=4) (actual time=0.005..0.010 rows=1 loops=1)
Filter: ((id = 1) OR (id = 5))
Rows Removed by Filter: 3
Planning time: 0.396 ms
Execution time: 0.139 ms
(9 rows)Can not detach / attach / drop partition table.
Best regards,
youngyonj1e.github.io
yangjie@highgo.com
--
Yugo Nagata <nagata@sraoss.co.jp>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Attachments:
hash_part_on_beta2_v1.patchapplication/octet-stream; name="=?UTF-8?Q?hash=5Fpart=5Fon=5Fbeta2=5Fv1.patch?="Download
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 011f2b9..a74081b 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1125,6 +1125,7 @@ relation_open(Oid relationId, LOCKMODE lockmode)
r = RelationIdGetRelation(relationId);
if (!RelationIsValid(r))
+
elog(ERROR, "could not open relation with OID %u", relationId);
/* Make note that we've accessed a temporary relation */
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 43b8924..8852b9c 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -48,6 +48,7 @@
#include "utils/rel.h"
#include "utils/ruleutils.h"
#include "utils/syscache.h"
+#include "utils/typcache.h"
/*
* Information about bounds of a partitioned relation
@@ -89,6 +90,7 @@ typedef struct PartitionBoundInfoData
* partitioned table) */
int null_index; /* Index of the null-accepting partition; -1
* if there isn't one */
+ int serial;
} PartitionBoundInfoData;
#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
@@ -114,6 +116,12 @@ typedef struct PartitionRangeBound
bool lower; /* this is the lower (vs upper) bound */
} PartitionRangeBound;
+/* Internal representation of a hash partition bound */
+typedef struct PartitionHashBound
+{
+ int serial; /* hash partition serial number */
+} PartitionHashBound;
+
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,
@@ -149,6 +157,49 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+/* check if the relation is a hash partition */
+bool is_hash_partition(Oid relid)
+{
+ return is_hash_partitioned(get_partition_parent(relid));
+}
+
+/* check if the relation is hash partitioned */
+bool is_hash_partitioned(Oid relid)
+{
+ Relation rel;
+ bool res = false;
+
+ rel = heap_open(relid, NoLock);
+ if (rel->rd_partkey)
+ if (rel->rd_partkey->strategy == 'h')
+ res = true;
+
+ heap_close(rel, NoLock);
+
+ return res;
+}
+
+/*
+ * HashSerialGetPartition
+ * get hash partition oid according to its index
+*/
+Oid
+HashSerialGetPartition(Oid parentOid, int index)
+{
+ Relation parent;
+ PartitionDesc pdesc;
+ Oid child;
+
+ parent = heap_open(parentOid, AccessShareLock);
+ pdesc = RelationGetPartitionDesc(parent);
+
+ child = pdesc->oids[index];
+
+ heap_close(parent, AccessShareLock);
+
+ return child;
+}
+
/*
* RelationBuildPartitionDesc
* Form rel's partition descriptor
@@ -179,6 +230,9 @@ RelationBuildPartitionDesc(Relation rel)
/* Range partitioning specific */
PartitionRangeBound **rbounds = NULL;
+ /* Hash partitiNG specific */
+ PartitionHashBound **hashserials = NULL;
+
/*
* The following could happen in situations where rel has a pg_class entry
* but not the pg_partitioned_table entry yet.
@@ -310,6 +364,10 @@ RelationBuildPartitionDesc(Relation rel)
qsort_arg(all_values, ndatums, sizeof(PartitionListValue *),
qsort_partition_list_value_cmp, (void *) key);
}
+ else if (key->strategy == PARTITION_STRATEGY_HASH)
+ {
+
+ }
else if (key->strategy == PARTITION_STRATEGY_RANGE)
{
int j,
@@ -511,6 +569,15 @@ RelationBuildPartitionDesc(Relation rel)
break;
}
+ case PARTITION_STRATEGY_HASH:
+ {
+ for (i = 0; i < nparts; i++)
+ {
+ mapping[i] = next_index++;
+ }
+ break;
+ }
+
case PARTITION_STRATEGY_RANGE:
{
boundinfo->content = (RangeDatumContent **) palloc(ndatums *
@@ -723,6 +790,13 @@ check_new_partition_bound(char *relname, Relation parent,
break;
}
+ case PARTITION_STRATEGY_HASH:
+ {
+ Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+
+ break;
+ }
+
case PARTITION_STRATEGY_RANGE:
{
PartitionRangeBound *lower,
@@ -898,6 +972,11 @@ get_qual_from_partbound(Relation rel, Relation parent,
my_qual = get_qual_for_range(key, spec);
break;
+ case PARTITION_STRATEGY_HASH:
+ Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+ my_qual = get_qual_for_range(key, spec);
+ break;
+
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
@@ -1039,6 +1118,7 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
/* Root partitioned table has no parent, so NULL for parent */
parted_rel_parents = list_make1(NULL);
APPEND_REL_PARTITION_OIDS(rel, all_parts, all_parents);
+
forboth(lc1, all_parts, lc2, all_parents)
{
Relation partrel = heap_open(lfirst_oid(lc1), lockmode);
@@ -1909,7 +1989,7 @@ FormPartitionKeyDatum(PartitionDispatch pd,
if (partexpr_item != NULL)
elog(ERROR, "wrong number of partition key expressions");
}
-
+
/*
* get_partition_for_tuple
* Finds a leaf partition for tuple contained in *slot
@@ -1935,6 +2015,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
+ int nparts;
/* start with the root partitioned table */
parent = pd[0];
@@ -2016,6 +2097,12 @@ get_partition_for_tuple(PartitionDispatch *pd,
cur_index = -1;
break;
+ case PARTITION_STRATEGY_HASH:
+ Assert(partdesc->nparts > 0);
+ nparts = partdesc->nparts;
+ cur_index = DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, values[0])) % nparts;
+
+ break;
case PARTITION_STRATEGY_RANGE:
/*
@@ -2257,6 +2344,13 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
*(Datum *) probe));
break;
+ case PARTITION_STRATEGY_HASH:
+ cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
+ key->partcollation[0],
+ bound_datums[0],
+ *(Datum *) probe));
+ break;
+
case PARTITION_STRATEGY_RANGE:
{
RangeDatumContent *content = boundinfo->content[offset];
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index bb00858..4e00a60 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -475,6 +475,11 @@ static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
PartitionCmd *cmd);
static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
+static void ReInsertHashPartition(Relation rel, Oid parentId, Node *bound);
+
+static void InsertIntoTable(Oid relid, HeapTuple tuple, TupleDesc tupdesc);
+static TupleTableSlot *ExecInsertPartition(TupleTableSlot *slot, TupleTableSlot *planSlot,
+ EState *estate, bool canSetTag);
/* ----------------------------------------------------------------
* DefineRelation
@@ -765,6 +770,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
/* Process and store partition bound, if any. */
if (stmt->partbound)
{
+
PartitionBoundSpec *bound;
ParseState *pstate;
Oid parentId = linitial_oid(inheritOids);
@@ -799,6 +805,9 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
/* Update the pg_class entry. */
StorePartitionBound(rel, parent, bound);
+ /*judge if create new hash partition, if so, re_insert data into all partitions*/
+ ReInsertHashPartition(rel, parentId, (Node *)bound);
+
heap_close(parent, NoLock);
/*
@@ -872,6 +881,210 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
return address;
}
+/*
+ * ReInsertHashPartition
+ * when create a new hash partition, re_insert data into all partitions
+ */
+static void
+ReInsertHashPartition(Relation rel, Oid parentId, Node *bound)
+{
+ Relation parent;
+ PartitionKey key;
+ PartitionDesc pdesc;
+ int nparts;
+
+ parent = heap_open(parentId, ExclusiveLock);
+ key = RelationGetPartitionKey(parent);
+ pdesc = RelationGetPartitionDesc(parent);
+
+ nparts = pdesc->nparts + 1;
+
+ if (key->strategy == PARTITION_STRATEGY_HASH)
+ {
+ int i;
+ Oid *children = pdesc->oids;
+
+ /*get data from every partition and re_insert*/
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ Oid child = children[i];
+ Relation hash_rel;
+ HeapScanDesc hash_scan;
+ HeapTuple hash_tup;
+ Snapshot snapshot;
+ Datum val;
+ bool isnull;
+ int index;
+ Oid insertRel;
+
+ hash_rel = heap_open(child, ExclusiveLock);
+ snapshot = RegisterSnapshot(GetLatestSnapshot());
+ hash_scan = heap_beginscan(hash_rel, snapshot, 0, NULL);
+
+ /*only partition table has tuples then re_insert*/
+ while ((hash_tup = heap_getnext(hash_scan, ForwardScanDirection)) != NULL)
+ {
+ /*get the key's data of the tuple*/
+ val = fastgetattr(hash_tup, key->partattrs[0], hash_rel->rd_att, &isnull);
+
+ /*get the partition's index of the key*/
+ index = DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, val)) % nparts;
+
+ /*get partition's oid that match the key's data*/
+ insertRel = HashSerialGetPartition(parentId, index);
+
+ if (index == nparts - 1)
+ {
+ InsertIntoTable(RelationGetRelid(rel), hash_tup, parent->rd_att);
+ simple_heap_delete(hash_rel, &hash_tup->t_self);
+ }
+ else
+ {
+ InsertIntoTable(insertRel, hash_tup, parent->rd_att);
+ simple_heap_delete(hash_rel, &hash_tup->t_self);
+ }
+ }
+
+ heap_endscan(hash_scan);
+ UnregisterSnapshot(snapshot);
+ heap_close(hash_rel, ExclusiveLock);
+ }
+ }
+
+ heap_close(parent, ExclusiveLock);
+}
+
+/*
+ * InsertIntoTable
+ * Find a table relation given it's name and insert the
+ * tuple on it (updating the indexes and calling the triggers)
+ */
+static void
+InsertIntoTable(Oid relid, HeapTuple tuple, TupleDesc tupdesc)
+{
+ ResultRelInfo *resultRelInfo;
+ TupleTableSlot *slot;
+ EState *estate = CreateExecutorState();
+ Relation relation;
+
+ /* Lookup the relation */
+ if (relid == InvalidOid)
+ elog(ERROR, "partition_insert_trigger: Invalid child table %s", get_rel_name(relid));
+ relation = RelationIdGetRelation(relid);
+ if (relation == NULL)
+ elog(ERROR, "partition_insert_trigger: Failed to locate relation for child table %s", get_rel_name(relid));
+
+ /*
+ * We need a ResultRelInfo so we can use the regular executor's
+ * index-entry-making machinery.
+ */
+ resultRelInfo = makeNode(ResultRelInfo);
+ resultRelInfo->ri_RangeTableIndex = 1; /* dummy */
+ resultRelInfo->ri_RelationDesc = relation;
+ resultRelInfo->ri_TrigDesc = CopyTriggerDesc(relation->trigdesc);
+ if (resultRelInfo->ri_TrigDesc)
+ {
+ resultRelInfo->ri_TrigFunctions = (FmgrInfo *)
+ palloc0(resultRelInfo->ri_TrigDesc->numtriggers * sizeof(FmgrInfo));
+ resultRelInfo->ri_TrigWhenExprs = (List **)
+ palloc0(resultRelInfo->ri_TrigDesc->numtriggers * sizeof(List *));
+ }
+ resultRelInfo->ri_TrigInstrument = NULL;
+ ExecOpenIndices(resultRelInfo, false);
+ estate->es_result_relations = resultRelInfo;
+ estate->es_num_result_relations = 1;
+ estate->es_result_relation_info = resultRelInfo;
+ /*added for GetModifiedColumns in ExecConstraints functions*/
+ estate->es_range_table = list_make1(relation);
+
+ /* Set up a tuple slot too */
+ slot = ExecInitExtraTupleSlot(estate);
+ ExecSetSlotDescriptor(slot, tupdesc);
+ /* Triggers might need a slot as well */
+ estate->es_trig_tuple_slot = ExecInitExtraTupleSlot(estate);
+
+ ExecStoreTuple(tuple, slot, InvalidBuffer, false);
+ ExecInsertPartition(slot, slot, estate, false);
+
+ /* Free resources */
+ ExecResetTupleTable(estate->es_tupleTable, false);
+ ExecCloseIndices(resultRelInfo);
+ FreeExecutorState(estate);
+ RelationClose(relation);
+}
+
+/*
+ * ExecInsertPartition:
+ * For INSERT, we have to insert the tuple into the target relation
+ * and insert appropriate tuples into the index relations.
+ */
+static TupleTableSlot *
+ExecInsertPartition(TupleTableSlot *slot, TupleTableSlot *planSlot, EState *estate, bool canSetTag)
+{
+ HeapTuple tuple;
+ ResultRelInfo *resultRelInfo;
+ Relation resultRelationDesc;
+ Oid newId;
+ List *recheckIndexes = NIL;
+
+ /*
+ * get the heap tuple out of the tuple table slot, making sure we have a
+ * writable copy
+ */
+ tuple = ExecMaterializeSlot(slot);
+
+ /* get information on the (current) result relation*/
+ resultRelInfo = estate->es_result_relation_info;
+ resultRelationDesc = resultRelInfo->ri_RelationDesc;
+
+ if (resultRelationDesc->rd_rel->relhasoids)
+ HeapTupleSetOid(tuple, InvalidOid);
+
+ /* BEFORE ROW INSERT Triggers */
+ if (resultRelInfo->ri_TrigDesc &&
+ resultRelInfo->ri_TrigDesc->trig_insert_before_row)
+ {
+ slot = ExecBRInsertTriggers(estate, resultRelInfo, slot);
+ if (slot == NULL)
+ return NULL;
+ /* trigger might have changed tuple */
+ tuple = ExecMaterializeSlot(slot);
+ }
+
+ /* INSTEAD OF ROW INSERT Triggers */
+ if (resultRelInfo->ri_TrigDesc &&
+ resultRelInfo->ri_TrigDesc->trig_insert_instead_row)
+ {
+ slot = ExecIRInsertTriggers(estate, resultRelInfo, slot);
+ if (slot == NULL)
+ return NULL;
+ /* trigger might have changed tuple */
+ tuple = ExecMaterializeSlot(slot);
+ newId = InvalidOid;
+ }
+ else
+ {
+ /* Check the constraints of the tuple*/
+ if (resultRelationDesc->rd_att->constr)
+ ExecConstraints(resultRelInfo, slot, estate);
+ newId = heap_insert(resultRelationDesc, tuple,
+ estate->es_output_cid, 0, NULL);
+ }
+
+ if (canSetTag)
+ {
+ (estate->es_processed)++;
+ estate->es_lastoid = newId;
+ setLastTid(&(tuple->t_self));
+ }
+
+ /* AFTER ROW INSERT Triggers */
+ ExecARInsertTriggers(estate, resultRelInfo, tuple, recheckIndexes, NULL);
+ list_free(recheckIndexes);
+
+ return NULL;
+}
+
/*
* Emit the right error or warning message for a "DROP" command issued on a
* non-existent relation
@@ -1166,6 +1379,15 @@ RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid, Oid oldRelOid,
LockRelationOid(state->heapOid, heap_lockmode);
}
+ if (is_partition && relOid != oldRelOid)
+ {
+ if(is_hash_partition(relOid))
+ ereport(ERROR,
+ (errcode(ERRCODE_WRONG_OBJECT_TYPE),
+ errmsg("hash partition \"%s\" can not be dropped",
+ get_rel_name(relOid))));
+ }
+
/*
* Similarly, if the relation is a partition, we must acquire lock on its
* parent before locking the partition. That's because queries lock the
@@ -1745,6 +1967,7 @@ MergeAttributes(List *schema, List *supers, char relpersistence,
* exclusive lock on the parent because its partition descriptor will
* be changed by addition of the new partition.
*/
+
if (!is_partition)
relation = heap_openrv(parent, ShareUpdateExclusiveLock);
else
@@ -13146,18 +13369,29 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
ParseState *pstate;
RangeTblEntry *rte;
ListCell *l;
-
+
newspec = makeNode(PartitionSpec);
newspec->strategy = partspec->strategy;
newspec->partParams = NIL;
newspec->location = partspec->location;
-
+
/* Parse partitioning strategy name */
if (pg_strcasecmp(partspec->strategy, "list") == 0)
*strategy = PARTITION_STRATEGY_LIST;
else if (pg_strcasecmp(partspec->strategy, "range") == 0)
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (pg_strcasecmp(partspec->strategy, "hash") == 0)
+ {
+ if (list_length(partspec->partParams) == 1)
+ {
+ *strategy = PARTITION_STRATEGY_HASH;
+ }
+ else
+ {
+ elog(ERROR, "The hash partition does not support multiple key values.");
+ }
+ }
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13434,6 +13668,13 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
ObjectAddress address;
const char *trigger_name;
+ /*hash partition do not support attach operation*/
+ if (rel->rd_partkey)
+ if (rel->rd_partkey->strategy == 'h')
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+ errmsg("hash partition do not support attach operation")));
+
attachRel = heap_openrv(cmd->name, AccessExclusiveLock);
/*
@@ -13771,6 +14012,13 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
new_repl[Natts_pg_class];
ObjectAddress address;
+ /*hash partition do not support detach operation*/
+ if (rel->rd_partkey)
+ if (rel->rd_partkey->strategy == 'h')
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+ errmsg("hash partition do not support detach operation")));
+
partRel = heap_openrv(name, AccessShareLock);
/* All inheritance related checks are performed within the function */
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0f08283..efd66bc 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -3228,6 +3228,7 @@ ExecSetupPartitionTupleRouting(Relation rel,
/* Get the tuple-routing information and lock partitions */
*pd = RelationGetPartitionDispatchInfo(rel, RowExclusiveLock, num_parted,
&leaf_parts);
+
*num_partitions = list_length(leaf_parts);
*partitions = (ResultRelInfo *) palloc(*num_partitions *
sizeof(ResultRelInfo));
@@ -3317,6 +3318,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
result = get_partition_for_tuple(pd, slot, estate,
&failed_at, &failed_slot);
+
if (result < 0)
{
Relation failed_rel;
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 8d17425..271b0f6 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -1919,6 +1919,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
&partition_tupconv_maps,
&partition_tuple_slot,
&num_parted, &num_partitions);
+
mtstate->mt_partition_dispatch_info = partition_dispatch_info;
mtstate->mt_num_dispatch = num_parted;
mtstate->mt_partitions = partitions;
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 67ac814..b86bfe6 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -24,6 +24,7 @@
#include "miscadmin.h"
#include "nodes/extensible.h"
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
#include "utils/datum.h"
@@ -4448,6 +4449,7 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
COPY_NODE_FIELD(listdatums);
COPY_NODE_FIELD(lowerdatums);
COPY_NODE_FIELD(upperdatums);
+ COPY_SCALAR_FIELD(hashnumber);
COPY_LOCATION_FIELD(location);
return newnode;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0abe9e..73d09cc 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3565,6 +3565,7 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_FIELD(listdatums);
WRITE_NODE_FIELD(lowerdatums);
WRITE_NODE_FIELD(upperdatums);
+ WRITE_INT_FIELD(hashnumber);
WRITE_LOCATION_FIELD(location);
}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 1380703..74391a7 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2377,6 +2377,7 @@ _readPartitionBoundSpec(void)
READ_NODE_FIELD(listdatums);
READ_NODE_FIELD(lowerdatums);
READ_NODE_FIELD(upperdatums);
+ READ_INT_FIELD(hashnumber);
READ_LOCATION_FIELD(location);
READ_DONE();
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index dc0b0b0..6333e43 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -28,7 +28,8 @@
#include "catalog/dependency.h"
#include "catalog/heap.h"
#include "catalog/partition.h"
-#include "catalog/pg_am.h"
+#include "catalog/pg_am.h"
+#include "catalog/pg_partitioned_table.h"
#include "catalog/pg_statistic_ext.h"
#include "foreign/fdwapi.h"
#include "miscadmin.h"
@@ -48,7 +49,7 @@
#include "utils/syscache.h"
#include "utils/rel.h"
#include "utils/snapmgr.h"
-
+#include "utils/typcache.h"
/* GUC parameter */
int constraint_exclusion = CONSTRAINT_EXCLUSION_PARTITION;
@@ -69,6 +70,10 @@ static List *build_index_tlist(PlannerInfo *root, IndexOptInfo *index,
Relation heapRelation);
static List *get_relation_statistics(RelOptInfo *rel, Relation relation);
+static int get_hash_part_attrs(Oid parentid);
+static char get_hash_part_strategy(Oid parentid);
+static int get_hash_part_number(Oid parentid);
+
/*
* get_relation_info -
* Retrieves catalog information for a given relation.
@@ -1441,9 +1446,376 @@ relation_excluded_by_constraints(PlannerInfo *root,
if (predicate_refuted_by(safe_constraints, rel->baserestrictinfo, false))
return true;
+ /* hash partition constraint exclusion. */
+ if (NIL != root->append_rel_list)
+ {
+ Node *parent = NULL;
+ parent = (Node*)linitial(root->append_rel_list);
+
+ if ((nodeTag(parent) == T_AppendRelInfo)
+ && get_hash_part_strategy(((AppendRelInfo*)parent)->parent_reloid) == PARTITION_STRATEGY_HASH
+ && (root->parse->jointree->quals != NULL))
+ {
+ int cur_index = -1, in_index = -1, bool_index = -1;
+ Oid poid;
+ Relation relation;
+ PartitionKey key;
+ RelabelType *rt;
+
+ poid = ((AppendRelInfo*)parent)->parent_reloid;
+ relation = RelationIdGetRelation(poid);
+ key = RelationGetPartitionKey(relation);
+
+ heap_close(relation, NoLock);
+
+ BoolExpr *boolexpr;
+ OpExpr *subopexpr;
+ Node *leftSubOp, *rightSubOp;
+ Node *constExprInSub, *varExprInSub;
+ Const *subc;
+ Var *subv;
+ int subpartattr;
+ int substrat;
+ TypeCacheEntry *subtce;
+
+ OpExpr *predicateExpr;
+ Node *predicate;
+ List *list;
+ Node *leftPredicateOp, *rightPredicateOp;
+ Node *constExprInPredicate, *varExprInPredicate;
+ Const *c;
+ Var *v;
+ int partattr;
+ int strat;
+ TypeCacheEntry *tce;
+
+ ScalarArrayOpExpr *arrayexpr;
+ Node *arraynode;
+
+ predicate = rel->baserestrictinfo;
+
+ list = (List *)predicate;
+
+ if (list_length(list) != 1)
+ return false;
+
+ predicate = linitial(list);
+ if (IsA(predicate, RestrictInfo))
+ {
+ RestrictInfo *info = (RestrictInfo*)predicate;
+ predicate = (Node*)info->clause;
+ }
+
+ switch (nodeTag(predicate))
+ {
+ /* AND, OR, NOT expressions */
+ case T_BoolExpr:
+
+ boolexpr = (BoolExpr *) predicate;
+ bool subflag = true;
+
+ foreach (lc, boolexpr->args)
+ {
+ Expr *arg = (Expr *) lfirst(lc);
+ OpExpr *oparg = (OpExpr*)lfirst(lc);
+
+ leftSubOp = get_leftop((Expr*)arg);
+ rightSubOp = get_rightop((Expr*)arg);
+ int *subidx;
+
+ /* check if one operand is a constant */
+ if (IsA(rightSubOp, Const))
+ {
+ varExprInSub = leftSubOp;
+ constExprInSub = rightSubOp;
+ }
+ else if (IsA(leftSubOp, Const))
+ {
+ constExprInSub = leftSubOp;
+ varExprInSub = rightSubOp;
+ }
+ else
+ {
+ return false;
+ }
+
+ subtce = lookup_type_cache(key->parttypid[0], TYPECACHE_BTREE_OPFAMILY);
+ substrat = get_op_opfamily_strategy(oparg->opno, subtce->btree_opf);
+
+ subpartattr = get_hash_part_attrs(poid);
+ subv = (Var*)varExprInSub;
+
+ switch (boolexpr->boolop)
+ {
+ case OR_EXPR:
+ if (subv->varattno == subpartattr && substrat == BTEqualStrategyNumber)
+ {
+ subc = (Const*)constExprInSub;
+ bool_index = DatumGetUInt32(OidFunctionCall1(
+ lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc,
+ subc->constvalue))
+ % list_length(root->append_rel_list);
+
+ subidx = &bool_index;
+ if (subidx != NULL)
+ {
+ if (get_hash_part_number(rte->relid) == *subidx)
+ {
+ subflag = false;
+ }
+
+ }
+ }
+ else
+ {
+ /* constraint exclusion in hash partition could not support <, > etc. */
+ return false;
+ }
+
+ break;
+
+ case AND_EXPR:
+
+ break;
+
+ default:
+
+ break;
+ }
+ }
+
+ if (subflag)
+ return true;
+ else
+ return false;
+
+ break;
+
+ /* =, !=, <, > etc. */
+ case T_OpExpr:
+ predicateExpr = (OpExpr *)predicate;
+ leftPredicateOp = get_leftop((Expr*)predicate);
+ rightPredicateOp = get_rightop((Expr*)predicate);
+
+ /* check if one operand is a constant */
+ if (IsA(rightPredicateOp, Const))
+ {
+ varExprInPredicate = leftPredicateOp;
+ constExprInPredicate = rightPredicateOp;
+ }
+ else if (IsA(leftPredicateOp, Const))
+ {
+ constExprInPredicate = leftPredicateOp;
+ varExprInPredicate = rightPredicateOp;
+ }
+ else
+ {
+ return false;
+ }
+
+ tce = lookup_type_cache(key->parttypid[0], TYPECACHE_BTREE_OPFAMILY);
+ strat = get_op_opfamily_strategy(predicateExpr->opno, tce->btree_opf);
+
+ partattr = get_hash_part_attrs(poid);
+ v = (Var*)varExprInPredicate;
+
+ /* If strat is "=", select one partiton */
+ if (v->varattno == partattr && strat == BTEqualStrategyNumber)
+ {
+ c = (Const*)constExprInPredicate;
+ cur_index = DatumGetUInt32(OidFunctionCall1(
+ lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc,
+ c->constvalue))
+ % list_length(root->append_rel_list);
+ }
+ else
+ {
+ /* constraint exclusion in hash partition could not support <, > etc. */
+ return false;
+ }
+
+ if (get_hash_part_number(rte->relid) != cur_index && cur_index != -1)
+ {
+ return true;
+ }
+
+ break;
+
+ /* IN expression */
+ case T_ScalarArrayOpExpr:
+ arrayexpr = (ScalarArrayOpExpr *)predicate;
+ arraynode = (Node *) lsecond(arrayexpr->args);
+ bool flag = true;
+
+ if (arraynode && IsA(arraynode, Const) &&
+ !((Const *) arraynode)->constisnull)
+ {
+ ArrayType *arrayval;
+ int16 elemlen;
+ bool elembyval;
+ char elemalign;
+ int num_elems;
+ Datum *elem_values;
+ bool *elem_nulls;
+ int strategy = BTEqualStrategyNumber;
+ int i;
+ int *idx;
+
+
+ /* Extract values from array */
+ arrayval = DatumGetArrayTypeP(((Const *) arraynode)->constvalue);
+ get_typlenbyvalalign(ARR_ELEMTYPE(arrayval),
+ &elemlen, &elembyval, &elemalign);
+ deconstruct_array(arrayval,
+ ARR_ELEMTYPE(arrayval),
+ elemlen, elembyval, elemalign,
+ &elem_values, &elem_nulls, &num_elems);
+
+ /* Construct OIDs list */
+ for (i = 0; i < num_elems; i++)
+ {
+ if (!elem_nulls[i])
+ {
+ /* Invoke base hash function for value type */
+ in_index = DatumGetUInt32(OidFunctionCall1(
+ lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc,
+ elem_values[i]))
+ % list_length(root->append_rel_list);
+ idx = &in_index;
+
+ if (idx != NULL)
+ {
+ if (get_hash_part_number(rte->relid) == *idx)
+ {
+ flag = false;
+ }
+
+ }
+ }
+ }
+
+ /* Free resources */
+ pfree(elem_values);
+ pfree(elem_nulls);
+ }
+
+ if (flag)
+ return true;
+ else
+ return false;
+
+ break;
+
+ default:
+ elog(ERROR, "Unknown clause type.");
+ }
+
+ }
+
+ return false;
+ }
+
return false;
}
+/*
+ * get_hash_part_attrs
+ *
+ */
+int
+get_hash_part_attrs(Oid parentid)
+{
+ Form_pg_partitioned_table form;
+ HeapTuple tuple;
+ int result;
+
+ tuple = SearchSysCache1(PARTRELID, ObjectIdGetDatum(parentid));
+
+ form = (Form_pg_partitioned_table)GETSTRUCT(tuple);
+ result = form->partattrs.values[0];
+
+ ReleaseSysCache(tuple);
+
+ return result;
+}
+
+/*
+ * get_hash_part_number
+ *
+ */
+int
+get_hash_part_number(Oid parentid)
+{
+ HeapTuple tuple;
+ int result;
+
+ tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(parentid));
+
+ Datum boundDatum;
+ bool isnull;
+ PartitionBoundSpec *bound;
+
+ boundDatum = SysCacheGetAttr(RELOID, tuple,
+ Anum_pg_class_relpartbound,
+ &isnull);
+
+ ReleaseSysCache(tuple);
+
+ if (boundDatum)
+ {
+ bound = (PartitionBoundSpec *)(stringToNode(TextDatumGetCString(boundDatum)));
+ return bound->hashnumber;
+ }
+
+ return result;
+}
+
+/*
+ * get_hash_part_strategy
+ *
+ */
+char
+get_hash_part_strategy(Oid parentid)
+{
+ Form_pg_partitioned_table form;
+ HeapTuple tuple;
+ char result;
+
+ tuple = SearchSysCache1(PARTRELID, ObjectIdGetDatum(parentid));
+ if (!HeapTupleIsValid(tuple))
+ {
+ tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(parentid));
+
+ if (!HeapTupleIsValid(tuple))
+ {
+ return '0';
+ }
+ else
+ {
+ Datum boundDatum;
+ bool isnull;
+ PartitionBoundSpec *bound;
+
+ boundDatum = SysCacheGetAttr(RELOID, tuple,
+ Anum_pg_class_relpartbound,
+ &isnull);
+
+ ReleaseSysCache(tuple);
+
+ if (boundDatum)
+ {
+ bound = (PartitionBoundSpec *)(stringToNode(TextDatumGetCString(boundDatum)));
+ return bound->strategy;
+ }
+ return '0';
+ }
+ }
+ form = (Form_pg_partitioned_table)GETSTRUCT(tuple);
+ result = form->partstrat;
+ ReleaseSysCache(tuple);
+
+ return result;
+}
/*
* build_physical_tlist
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 0f3998f..1575cbe 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -2675,6 +2675,17 @@ ForValues:
$$ = n;
}
+ /* a HASH partition */
+ | /*EMPTY*/
+ {
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ //n->hashnumber = 1;
+ //n->location = @3;
+
+ $$ = n;
+ }
;
partbound_datum:
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index ee5f3a3..1742fb3 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -33,6 +33,7 @@
#include "catalog/heap.h"
#include "catalog/index.h"
#include "catalog/namespace.h"
+#include "catalog/partition.h"
#include "catalog/pg_am.h"
#include "catalog/pg_collation.h"
#include "catalog/pg_constraint.h"
@@ -3299,6 +3300,8 @@ transformPartitionBound(ParseState *pstate, Relation parent,
char strategy = get_partition_strategy(key);
int partnatts = get_partition_natts(key);
List *partexprs = get_partition_exprs(key);
+ PartitionDesc pdesc;
+ pdesc = RelationGetPartitionDesc(parent);
/* Avoid scribbling on input */
result_spec = copyObject(spec);
@@ -3359,6 +3362,28 @@ transformPartitionBound(ParseState *pstate, Relation parent,
value);
}
}
+ else if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ char *colname;
+
+ 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))));
+
+ /* Get the only column's name in case we need to output an error */
+ if (key->partattrs[0] != 0)
+ colname = get_relid_attribute_name(RelationGetRelid(parent),
+ key->partattrs[0]);
+ else
+ colname = deparse_expression((Node *) linitial(partexprs),
+ deparse_context_for(RelationGetRelationName(parent),
+ RelationGetRelid(parent)),
+ false, false);
+
+ result_spec->hashnumber = pdesc->nparts;
+ }
else if (strategy == PARTITION_STRATEGY_RANGE)
{
ListCell *cell1,
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 18d9e27..19d216d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1660,6 +1660,10 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags,
if (!attrsOnly)
appendStringInfo(&buf, "RANGE");
break;
+ case PARTITION_STRATEGY_HASH:
+ if (!attrsOnly)
+ appendStringInfo(&buf, "HASH");
+ break;
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) form->partstrat);
@@ -8666,6 +8670,13 @@ get_rule_expr(Node *node, deparse_context *context,
appendStringInfoString(buf, ")");
break;
+ case PARTITION_STRATEGY_HASH:
+ Assert(spec->hashnumber != -1);
+
+ appendStringInfoString(buf, "SERIAL NUMBER");
+ appendStringInfo(buf, " %d", spec->hashnumber);
+ break;
+
case PARTITION_STRATEGY_RANGE:
Assert(spec->lowerdatums != NIL &&
spec->upperdatums != NIL &&
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 502d5eb..4d71c21 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -15340,12 +15340,33 @@ dumpTableSchema(Archive *fout, TableInfo *tbinfo)
appendPQExpBufferStr(q, " (\n)");
}
- if (tbinfo->ispartition && !dopt->binary_upgrade)
+ if(tbinfo->ispartition && !dopt->binary_upgrade)
{
- appendPQExpBufferStr(q, "\n");
- appendPQExpBufferStr(q, tbinfo->partbound);
+ PGresult *res;
+ PQExpBuffer query = createPQExpBuffer();
+ char *strategy;
+ char *s = "h";
+
+ appendPQExpBuffer(query, " select partstrat"
+ " from pg_partitioned_table p"
+ " join pg_inherits i on i.inhparent = p.partrelid"
+ " where i.inhrelid = %d", tbinfo->dobj.catId.oid);
+
+ res = ExecuteSqlQueryForSingleRow(fout, query->data);
+ strategy = pg_strdup(PQgetvalue(res, 0, 0));
+
+ if(!(strcmp(strategy, s) == 0))
+ {
+ appendPQExpBufferStr(q, "\n");
+ appendPQExpBufferStr(q, tbinfo->partbound);
+ }
+
+ PQclear(res);
+ destroyPQExpBuffer(query);
}
+
+
/* Emit the INHERITS clause, except if this is a partition. */
if (numParents > 0 &&
!tbinfo->ispartition &&
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index f10879a..ecdd757 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -70,6 +70,10 @@ typedef struct PartitionDispatchData
typedef struct PartitionDispatchData *PartitionDispatch;
+extern bool is_hash_partitioned(Oid relid);
+extern bool is_hash_partition(Oid relid);
+extern Oid HashSerialGetPartition(Oid parentOid, int index);
+
extern void RelationBuildPartitionDesc(Relation relation);
extern bool partition_bounds_equal(PartitionKey key,
PartitionBoundInfo p1, PartitionBoundInfo p2);
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 1d96169..73b8281 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -785,6 +785,7 @@ typedef struct PartitionSpec
/* Internal codes for partitioning strategies */
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'
/*
* PartitionBoundSpec - a partition bound specification
@@ -805,6 +806,9 @@ typedef struct PartitionBoundSpec
List *lowerdatums; /* List of PartitionRangeDatums */
List *upperdatums; /* List of PartitionRangeDatums */
+ /* Partitioning info for HASH strategy: */
+ int hashnumber; /* serial number */
+
int location; /* token location, or -1 if unknown */
} PartitionBoundSpec;
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index b6f794e..e434ae2 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index cb7aa5b..95b0e23 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
I've updated patch to use an extended hash function (Commit #
81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.
Regards,
Amul
On Thu, Jul 27, 2017 at 5:11 PM, amul sul <sulamul@gmail.com> wrote:
Show quoted text
Attaching newer patches rebased against the latest master head. Thanks !
Regards,
Amul
Attachments:
0001-Cleanup_v6.patchapplication/octet-stream; name=0001-Cleanup_v6.patchDownload
From d8473beed57404d1bd99a6cde3ccae36c9dad6f2 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 3 Jul 2017 15:07:33 +0530
Subject: [PATCH 1/2] Cleanup_v6
Code refactoring required for hash partitioning patch v17.
---
src/backend/catalog/partition.c | 103 ++++++++++++++++++++++++----------------
1 file changed, 61 insertions(+), 42 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 96a64ce..b247f02 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1940,9 +1940,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
PartitionDispatch parent;
Datum values[PARTITION_MAX_KEYS];
bool isnull[PARTITION_MAX_KEYS];
- int cur_offset,
- cur_index;
- int i,
+ int cur_index,
result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1986,48 +1984,69 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /*
- * Since we cannot route tuples with NULL partition keys through a
- * range-partitioned table, simply return that no partition exists
- */
- for (i = 0; i < key->partnatts; i++)
- {
- if (isnull[i])
- {
- *failed_at = parent;
- *failed_slot = slot;
- result = -1;
- goto error_exit;
- }
- }
- }
-
/*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
+ * cur_index should be reset at every iteration here. Otherwise, we
+ * might have hazardous array subscript.
*/
cur_index = -1;
- if (isnull[0] && partition_bound_accepts_nulls(partdesc->boundinfo))
- cur_index = partdesc->boundinfo->null_index;
- else if (!isnull[0])
+
+ switch (key->strategy)
{
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
+ case PARTITION_STRATEGY_LIST:
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partition_bound_accepts_nulls(partdesc->boundinfo))
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
+
+ /* bsearch in partdesc->boundinfo */
+ cur_offset = partition_bound_bsearch(key,
+ partdesc->boundinfo,
+ values,
+ false,
+ &equal);
if (cur_offset >= 0 && equal)
cur_index = partdesc->boundinfo->indexes[cur_offset];
- else
- cur_index = -1;
- break;
+ }
+ break;
+
+ case PARTITION_STRATEGY_RANGE:
+ {
+ bool equal = false;
+ int cur_offset;
+ int i;
+
+ /*
+ * Since we cannot route tuples with NULL partition keys
+ * through a range-partitioned table, simply return that
+ * no partition exists
+ */
+ for (i = 0; i < key->partnatts; i++)
+ {
+ if (isnull[i])
+ {
+ *failed_at = parent;
+ *failed_slot = slot;
+ result = -1;
+ goto error_exit;
+ }
+ }
- case PARTITION_STRATEGY_RANGE:
+ /* bsearch in partdesc->boundinfo */
+ cur_offset = partition_bound_bsearch(key,
+ partdesc->boundinfo,
+ values,
+ false,
+ &equal);
/*
* Offset returned is such that the bound at offset is
@@ -2035,12 +2054,12 @@ get_partition_for_tuple(PartitionDispatch *pd,
* at offset+1 would be the upper bound.
*/
cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
- break;
+ }
+ break;
- default:
- elog(ERROR, "unexpected partition strategy: %d",
- (int) key->strategy);
- }
+ default:
+ elog(ERROR, "unexpected partition strategy: %d",
+ (int) key->strategy);
}
/*
--
2.6.2
0002-hash-partitioning_another_design-v17.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v17.patchDownload
From e77262b5bef02c0b70df39c3ebfb02ef49079544 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Wed, 23 Aug 2017 15:19:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v17
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 77 +++-
src/backend/catalog/partition.c | 586 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 75 +++-
src/backend/parser/parse_utilcmd.c | 22 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 60 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 43 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1170 insertions(+), 95 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2..24b36ca 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying modulus and remainder for each
+ partition. Each partition holds rows for which the hash value of
+ partition keys when divided by specified modulus produces specified
+ remainder. For more clarification on modulus and remainder please refer
+ <xref linkend="sql-createtable-partition">.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 6960032..7b60927 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1396,6 +1396,13 @@ ALTER TABLE cities
</programlisting></para>
<para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (modulus 4, remainder 3);
+</programlisting></para>
+
+ <para>
Detach a partition from partitioned table:
<programlisting>
ALTER TABLE measurement
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index e9c2c49..043bb87 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable>
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -343,6 +344,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</para>
<para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the children of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 children, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
+ <para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
OIDS</literal> then all partitions must have OIDs; the parent's OID
@@ -465,7 +489,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -476,9 +500,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can
altered when building <productname>PostgreSQL</productname>.), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provide only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1629,6 +1667,16 @@ CREATE TABLE cities (
</programlisting></para>
<para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
+</programlisting></para>
+
+ <para>
Create partition of a range partitioned table:
<programlisting>
CREATE TABLE measurement_y2016m07
@@ -1679,6 +1727,19 @@ CREATE TABLE cities_ab
CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
</refsect1>
<refsect1 id="SQL-CREATETABLE-compatibility">
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b247f02..75d65f0 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -58,26 +59,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 */
} PartitionBoundInfoData;
@@ -89,6 +99,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
{
@@ -105,6 +123,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,
@@ -120,12 +139,15 @@ 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(PartitionKey key, PartitionBoundSpec *spec);
static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
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);
@@ -139,6 +161,10 @@ static int32 partition_bound_cmp(PartitionKey key,
static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+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
@@ -163,6 +189,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -228,7 +257,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;
@@ -452,6 +509,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));
@@ -562,8 +655,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];
@@ -597,49 +689,97 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
if (b1->null_index != b2->null_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 or
+ * same moduli with different number of partitions, 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. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -661,6 +801,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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);
@@ -878,6 +1101,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(key, spec);
+ break;
+
case PARTITION_STRATEGY_LIST:
Assert(spec->strategy == PARTITION_STRATEGY_LIST);
my_qual = get_qual_for_list(key, spec);
@@ -1279,6 +1507,123 @@ make_partition_op_expr(PartitionKey key, int keynum,
}
/*
+ * 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 2, remainder 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 4, remainder 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns an implicit-AND list of expressions to use as a list partition's
@@ -1992,6 +2337,19 @@ get_partition_for_tuple(PartitionDispatch *pd,
switch (key->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ {
+ PartitionBoundInfo boundinfo = partdesc->boundinfo;
+ int ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2089,6 +2447,38 @@ error_exit:
}
/*
+ * 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
*
* Compare two list partition bound datums
@@ -2272,6 +2662,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],
@@ -2355,3 +2754,104 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 0f08245..bb9b0cf 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -468,7 +468,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,
@@ -849,7 +849,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);
@@ -13187,7 +13187,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;
@@ -13257,10 +13259,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)
@@ -13420,25 +13424,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f9ddf4e..48c595c 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4449,6 +4449,8 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
PartitionBoundSpec *newnode = makeNode(PartitionBoundSpec);
COPY_SCALAR_FIELD(strategy);
+ COPY_SCALAR_FIELD(modulus);
+ COPY_SCALAR_FIELD(remainder);
COPY_NODE_FIELD(listdatums);
COPY_NODE_FIELD(lowerdatums);
COPY_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8d92c03..59b90c2 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2838,6 +2838,8 @@ static bool
_equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *b)
{
COMPARE_SCALAR_FIELD(strategy);
+ COMPARE_SCALAR_FIELD(modulus);
+ COMPARE_SCALAR_FIELD(remainder);
COMPARE_NODE_FIELD(listdatums);
COMPARE_NODE_FIELD(lowerdatums);
COMPARE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 9ee3e23..f78eb0b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3573,6 +3573,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUNDSPEC");
WRITE_CHAR_FIELD(strategy);
+ WRITE_INT_FIELD(modulus);
+ WRITE_INT_FIELD(remainder);
WRITE_NODE_FIELD(listdatums);
WRITE_NODE_FIELD(lowerdatums);
WRITE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 67b9e19..df3a84c 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2390,6 +2390,8 @@ _readPartitionBoundSpec(void)
READ_LOCALS(PartitionBoundSpec);
READ_CHAR_FIELD(strategy);
+ READ_INT_FIELD(modulus);
+ READ_INT_FIELD(remainder);
READ_NODE_FIELD(listdatums);
READ_NODE_FIELD(lowerdatums);
READ_NODE_FIELD(upperdatums);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 7d0de99..713a548 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -577,7 +577,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <list> part_params
%type <partboundspec> ForValues
%type <node> partbound_datum PartitionRangeDatum
-%type <list> partbound_datum_list range_datum_list
+%type <list> partbound_datum_list range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2620,8 +2621,60 @@ alter_identity_column_option:
;
ForValues:
+ /* 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);
@@ -2646,6 +2699,24 @@ ForValues:
}
;
+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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 2058679..72b6ad3 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3307,7 +3307,27 @@ transformPartitionBound(ParseState *pstate, Relation parent,
/* Avoid scribbling on input */
result_spec = copyObject(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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 43646d2..0c0967a 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8701,6 +8705,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e3780..c58d1e1 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 1583cfa..ae8ed11 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2052,7 +2052,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_CONST("FOR VALUES");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2283c67..2b484d5 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d820b56..732ad45 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5529,6 +5529,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 5f2a4a7..de8ea28 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -798,6 +800,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index ed03cb9..4176642 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3273,6 +3273,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check validation when attaching list partitions
CREATE TABLE list_parted2 (
a int,
@@ -3402,6 +3403,59 @@ DETAIL: "part_5" is already a child of "list_parted2".
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
ERROR: circular inheritance not allowed
DETAIL: "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3413,12 +3467,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3514,6 +3573,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index babda89..fa0cd35 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (modu...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -505,6 +505,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -514,6 +519,30 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -521,6 +550,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -603,6 +634,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -751,6 +799,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index e159d62..46dae0a 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -314,8 +314,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table mlparted (a int, b int) partition by range (a, b);
create table mlparted1 (b int not null, a int not null) partition by range ((b+0));
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index 9366f04..eeed09f 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,34 @@ ERROR: new row for relation "part_b_10_b_20" violates partition constraint
DETAIL: Failing row contains (b, 9).
-- ok
update range_parted set b = b + 1 where b = 10;
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 9a20dd1..1a40ca2 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2095,6 +2095,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check validation when attaching list partitions
CREATE TABLE list_parted2 (
@@ -2223,6 +2224,62 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2234,12 +2291,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2311,6 +2372,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index 1c0ce927..70c3cde 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
@@ -477,6 +474,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -485,6 +484,22 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+-- trying to specify range for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -492,6 +507,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -565,6 +581,21 @@ CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1,
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -634,6 +665,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index 6f17872..3cfcaeb 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table mlparted (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 6637119..75d16fc 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,33 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 17ba2bd..6d52f84 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.6.2
On Mon, Sep 4, 2017 at 4:08 PM, amul sul <sulamul@gmail.com> wrote:
I've updated patch to use an extended hash function (Commit #
81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.I have done some testing with these patches, everything looks fine,
attaching sql and out file for reference.
Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation
On Mon, Sep 4, 2017 at 6:38 AM, amul sul <sulamul@gmail.com> wrote:
I've updated patch to use an extended hash function (Commit #
81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.
Committed 0001 after noticing that Jeevan Ladhe also found that change
convenient for default partitioning. I made a few minor cleanups;
hopefully I didn't break anything.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Sep 8, 2017 at 6:45 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Sep 4, 2017 at 6:38 AM, amul sul <sulamul@gmail.com> wrote:
I've updated patch to use an extended hash function (Commit #
81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.Committed 0001 after noticing that Jeevan Ladhe also found that change
convenient for default partitioning. I made a few minor cleanups;
hopefully I didn't break anything.
Thanks you.
Rebased 0002 against this commit & renamed to 0001, PFA.
Regards,
Amul
Attachments:
0001-hash-partitioning_another_design-v18.patchapplication/octet-stream; name=0001-hash-partitioning_another_design-v18.patchDownload
From 95f93f80f5e0963809becaa820087501a6c1c501 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 8 Sep 2017 17:47:40 +0530
Subject: [PATCH] hash-partitioning_another_design-v18
v18:
0001-Cleanup_v6.patch got committed(f0a0c17).
Rebased against patch against latest head.
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 77 +++-
src/backend/catalog/partition.c | 586 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 75 +++-
src/backend/parser/parse_utilcmd.c | 22 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 60 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 43 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1170 insertions(+), 95 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2..24b36ca 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying modulus and remainder for each
+ partition. Each partition holds rows for which the hash value of
+ partition keys when divided by specified modulus produces specified
+ remainder. For more clarification on modulus and remainder please refer
+ <xref linkend="sql-createtable-partition">.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index dae6307..016c724 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1396,6 +1396,13 @@ ALTER TABLE cities
</programlisting></para>
<para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (modulus 4, remainder 3);
+</programlisting></para>
+
+ <para>
Detach a partition from partitioned table:
<programlisting>
ALTER TABLE measurement
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index a6ca590..6148191 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable>
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -343,6 +344,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</para>
<para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the children of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 children, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
+ <para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
OIDS</literal> then all partitions must have OIDs; the parent's OID
@@ -465,7 +489,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -476,9 +500,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provide only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1629,6 +1667,16 @@ CREATE TABLE cities (
</programlisting></para>
<para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
+</programlisting></para>
+
+ <para>
Create partition of a range partitioned table:
<programlisting>
CREATE TABLE measurement_y2016m07
@@ -1679,6 +1727,19 @@ CREATE TABLE cities_ab
CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
</refsect1>
<refsect1 id="SQL-CREATETABLE-compatibility">
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index c6bd02f..fb1f0e4 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -58,26 +59,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 */
} PartitionBoundInfoData;
@@ -89,6 +99,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
{
@@ -105,6 +123,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,
@@ -120,12 +139,15 @@ 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(PartitionKey key, PartitionBoundSpec *spec);
static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
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);
@@ -139,6 +161,10 @@ static int32 partition_bound_cmp(PartitionKey key,
static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+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
@@ -163,6 +189,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -228,7 +257,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;
@@ -432,6 +489,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));
@@ -542,8 +635,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];
@@ -577,49 +669,97 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
if (b1->null_index != b2->null_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 or
+ * same moduli with different number of partitions, 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. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -641,6 +781,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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);
@@ -858,6 +1081,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(key, spec);
+ break;
+
case PARTITION_STRATEGY_LIST:
Assert(spec->strategy == PARTITION_STRATEGY_LIST);
my_qual = get_qual_for_list(key, spec);
@@ -1259,6 +1487,123 @@ make_partition_op_expr(PartitionKey key, int keynum,
}
/*
+ * 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 2, remainder 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 4, remainder 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns an implicit-AND list of expressions to use as a list partition's
@@ -1967,6 +2312,19 @@ 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 ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
if (isnull[0])
@@ -2054,6 +2412,38 @@ error_exit:
}
/*
+ * 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
*
* Compare two list partition bound datums
@@ -2237,6 +2627,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],
@@ -2320,3 +2719,104 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index c8fc9cb..d4dd91a 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -468,7 +468,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,
@@ -849,7 +849,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);
@@ -13218,7 +13218,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;
@@ -13288,10 +13290,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)
@@ -13451,25 +13455,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 9bae264..5036e2b 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4450,6 +4450,8 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
PartitionBoundSpec *newnode = makeNode(PartitionBoundSpec);
COPY_SCALAR_FIELD(strategy);
+ COPY_SCALAR_FIELD(modulus);
+ COPY_SCALAR_FIELD(remainder);
COPY_NODE_FIELD(listdatums);
COPY_NODE_FIELD(lowerdatums);
COPY_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 11731da..4c5c1fa 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2839,6 +2839,8 @@ static bool
_equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *b)
{
COMPARE_SCALAR_FIELD(strategy);
+ COMPARE_SCALAR_FIELD(modulus);
+ COMPARE_SCALAR_FIELD(remainder);
COMPARE_NODE_FIELD(listdatums);
COMPARE_NODE_FIELD(lowerdatums);
COMPARE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 9ee3e23..f78eb0b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3573,6 +3573,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUNDSPEC");
WRITE_CHAR_FIELD(strategy);
+ WRITE_INT_FIELD(modulus);
+ WRITE_INT_FIELD(remainder);
WRITE_NODE_FIELD(listdatums);
WRITE_NODE_FIELD(lowerdatums);
WRITE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 67b9e19..df3a84c 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2390,6 +2390,8 @@ _readPartitionBoundSpec(void)
READ_LOCALS(PartitionBoundSpec);
READ_CHAR_FIELD(strategy);
+ READ_INT_FIELD(modulus);
+ READ_INT_FIELD(remainder);
READ_NODE_FIELD(listdatums);
READ_NODE_FIELD(lowerdatums);
READ_NODE_FIELD(upperdatums);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 5eb3981..f86ded9 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -577,7 +577,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <list> part_params
%type <partboundspec> ForValues
%type <node> partbound_datum PartitionRangeDatum
-%type <list> partbound_datum_list range_datum_list
+%type <list> partbound_datum_list range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2636,8 +2637,60 @@ alter_identity_column_option:
;
ForValues:
+ /* 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);
@@ -2662,6 +2715,24 @@ ForValues:
}
;
+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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 2058679..72b6ad3 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3307,7 +3307,27 @@ transformPartitionBound(ParseState *pstate, Relation parent,
/* Avoid scribbling on input */
result_spec = copyObject(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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index f9ea7ed..557c99e 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8701,6 +8705,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e3780..c58d1e1 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 2ab8809..7a5fc26 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_CONST("FOR VALUES");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2283c67..2b484d5 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index d820b56..732ad45 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5529,6 +5529,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 3171815..f94653f 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -798,6 +800,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 0f36423..3091b85 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check validation when attaching list partitions
CREATE TABLE list_parted2 (
a int,
@@ -3426,6 +3427,59 @@ DETAIL: "part_5" is already a child of "list_parted2".
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
ERROR: circular inheritance not allowed
DETAIL: "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3437,12 +3491,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3538,6 +3597,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index babda89..fa0cd35 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (modu...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -505,6 +505,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -514,6 +519,30 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -521,6 +550,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -603,6 +634,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -751,6 +799,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index e159d62..46dae0a 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -314,8 +314,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table mlparted (a int, b int) partition by range (a, b);
create table mlparted1 (b int not null, a int not null) partition by range ((b+0));
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index 9366f04..eeed09f 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,34 @@ ERROR: new row for relation "part_b_10_b_20" violates partition constraint
DETAIL: Failing row contains (b, 9).
-- ok
update range_parted set b = b + 1 where b = 10;
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index e6f6669..4c3a153 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check validation when attaching list partitions
CREATE TABLE list_parted2 (
@@ -2239,6 +2240,62 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2250,12 +2307,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2327,6 +2388,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index 1c0ce927..70c3cde 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
@@ -477,6 +474,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -485,6 +484,22 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+-- trying to specify range for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -492,6 +507,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -565,6 +581,21 @@ CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1,
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -634,6 +665,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index 6f17872..3cfcaeb 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table mlparted (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 6637119..75d16fc 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,33 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 17ba2bd..6d52f84 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.6.2
On Fri, Sep 8, 2017 at 6:10 PM, amul sul <sulamul@gmail.com> wrote:
On Fri, Sep 8, 2017 at 6:45 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Sep 4, 2017 at 6:38 AM, amul sul <sulamul@gmail.com> wrote:
I've updated patch to use an extended hash function (Commit #
81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.Committed 0001 after noticing that Jeevan Ladhe also found that change
convenient for default partitioning. I made a few minor cleanups;
hopefully I didn't break anything.Thanks you.
Rebased 0002 against this commit & renamed to 0001, PFA.
Given that we have default partition support now, I am wondering
whether hash partitioned tables also should have default partitions.
The way we have structured hash partitioning syntax, there can be
"holes" in partitions. Default partition would help plug those holes.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Rebased 0002 against this commit & renamed to 0001, PFA.
Given that we have default partition support now, I am wondering
whether hash partitioned tables also should have default partitions.
The way we have structured hash partitioning syntax, there can be
"holes" in partitions. Default partition would help plug those holes.
Yeah, I was thinking about that, too. On the one hand, it seems like
it's solving the problem the wrong way: if you've set up hash
partitioning properly, you shouldn't have any holes. On the other
hand, supporting it probably wouldn't cost anything noticeable and
might make things seem more consistent. I'm not sure which way to
jump on this one.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Robert Haas wrote:
On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:Rebased 0002 against this commit & renamed to 0001, PFA.
Given that we have default partition support now, I am wondering
whether hash partitioned tables also should have default partitions.
The way we have structured hash partitioning syntax, there can be
"holes" in partitions. Default partition would help plug those holes.Yeah, I was thinking about that, too. On the one hand, it seems like
it's solving the problem the wrong way: if you've set up hash
partitioning properly, you shouldn't have any holes. On the other
hand, supporting it probably wouldn't cost anything noticeable and
might make things seem more consistent. I'm not sure which way to
jump on this one.
How difficult/tedious/troublesome would be to install the missing
partitions if you set hash partitioning with a default partition and
only later on notice that some partitions are missing? I think if the
answer is that you need to exclusive-lock something for a long time and
this causes a disruption in production systems, then it's better not to
allow a default partition at all and just force all the hash partitions
to be there from the start.
On the other hand, if you can get tuples out of the default partition
into their intended regular partitions without causing any disruption,
then it seems okay to allow default partitions in hash partitioning
setups.
(I, like many others, was unable to follow the default partition stuff
as closely as I would have liked.)
--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Sep 11, 2017 at 8:00 AM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:
How difficult/tedious/troublesome would be to install the missing
partitions if you set hash partitioning with a default partition and
only later on notice that some partitions are missing? I think if the
answer is that you need to exclusive-lock something for a long time and
this causes a disruption in production systems, then it's better not to
allow a default partition at all and just force all the hash partitions
to be there from the start.On the other hand, if you can get tuples out of the default partition
into their intended regular partitions without causing any disruption,
then it seems okay to allow default partitions in hash partitioning
setups.
I think there's no real use case for default partitioning, and yeah,
you do need exclusive locks to repartition things (whether hash
partitioning or otherwise). It would be nice to fix that eventually,
but it's hard, because the executor has to cope with the floor moving
under it, and as of today, it really can't cope with that at all - not
because of partitioning specifically, but because of existing design
decisions that will require a lot of work (and probably arguing) to
revisit.
I think the way to get around the usability issues for hash
partitioning is to eventually add some syntax that does things like
(1) automatically create the table with N properly-configured
partitions, (2) automatically split an existing partition into N
pieces, and (3) automatically rewrite the whole table using a
different partition count.
People seem to find the hash partitioning stuff a little arcane. I
don't want to discount that confusion with some sort of high-handed "I
know better" attitude, I think the interface that users will actually
see can end up being pretty straightforward. The complexity that is
there in the syntax is to allow pg_upgrade and pg_dump/restore to work
properly. But users don't necessarily have to use the same syntax
that pg_dump does, just as you can say CREATE INDEX ON a (b) and let
the system specify the index name, but at dump time the index name is
specified explicitly.
(I, like many others, was unable to follow the default partition stuff
as closely as I would have liked.)
Uh, sorry about that. Would it help if I wrote a blog post on it or
something? The general idea is simple: any tuples that don't route to
any other partition get routed to the default partition.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Sep 11, 2017 at 5:30 PM, Alvaro Herrera <alvherre@alvh.no-ip.org>
wrote:
Robert Haas wrote:
On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:Rebased 0002 against this commit & renamed to 0001, PFA.
Given that we have default partition support now, I am wondering
whether hash partitioned tables also should have default partitions.
The way we have structured hash partitioning syntax, there can be
"holes" in partitions. Default partition would help plug those holes.Yeah, I was thinking about that, too. On the one hand, it seems like
it's solving the problem the wrong way: if you've set up hash
partitioning properly, you shouldn't have any holes. On the other
hand, supporting it probably wouldn't cost anything noticeable and
might make things seem more consistent. I'm not sure which way to
jump on this one.How difficult/tedious/troublesome would be to install the missing
partitions if you set hash partitioning with a default partition and
only later on notice that some partitions are missing? I think if the
answer is that you need to exclusive-lock something for a long time and
this causes a disruption in production systems, then it's better not to
allow a default partition at all and just force all the hash partitions
to be there from the start.
I am also leaning toward not to support a default partition for a hash
partitioned table.
The major drawback I can see is the constraint get created on the default
partition
table. IIUC, constraint on the default partition table are just negation
of partition
constraint on all its sibling partitions.
Consider a hash partitioned table having partitions with (modulus 64,
remainder 0) ,
...., (modulus 64, remainder 62) hash bound and partition column are col1,
col2,...,so on,
then constraint for the default partition will be :
NOT( (satisfies_hash_partition(64, 0, hash_fn1(col1), hash_fn2(col2), ...)
&& ... &&
satisfies_hash_partition(64, 62, hash_fn1(col1),hash_fn2(col2), ...))
Which will be much harmful to the performance than any other partitioning
strategy because it calculate a hash for the same partitioning key multiple
time.
We could overcome this by having an another SQL function (e.g
satisfies_default_hash_partition)
which calculates hash value once and checks the remainder, and that would be
a different path from the current default partition framework.
Regards,
Amul
Hi Amul,
On 09/08/2017 08:40 AM, amul sul wrote:
Rebased 0002 against this commit & renamed to 0001, PFA.
This patch needs a rebase.
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen <jesper.pedersen@redhat.com
wrote:
Hi Amul,
On 09/08/2017 08:40 AM, amul sul wrote:
Rebased 0002 against this commit & renamed to 0001, PFA.
This patch needs a rebase.
Thanks for your note.
Attached is the patch rebased on the latest master head.
Also added error on
creating
d
efault partition
for the hash partitioned table
,
and updated document &
test script for the same.
Regards,
Amul
Attachments:
0001-hash-partitioning_another_design-v19.patchapplication/octet-stream; name=0001-hash-partitioning_another_design-v19.patchDownload
From 2293ed78a9b5b90f680ec9e671ac47068803ab9f Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Thu, 14 Sep 2017 13:20:45 +0530
Subject: [PATCH] hash-partitioning_another_design-v19
v19:
Rebased on latest head.
Added error and test case for a default hash partition.
v18:
0001-Cleanup_v6.patch got committed(f0a0c17).
Rebased against patch against latest head.
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 590 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1188 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2..24b36ca 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying modulus and remainder for each
+ partition. Each partition holds rows for which the hash value of
+ partition keys when divided by specified modulus produces specified
+ remainder. For more clarification on modulus and remainder please refer
+ <xref linkend="sql-createtable-partition">.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385e..a6eefb8 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1421,6 +1421,13 @@ ALTER TABLE cities
</programlisting></para>
<para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (modulus 4, remainder 3);
+</programlisting></para>
+
+ <para>
Detach a partition from partitioned table:
<programlisting>
ALTER TABLE measurement
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 824253d..7e57e22 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -365,6 +367,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</para>
<para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the children of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 children, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
+ <para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
OIDS</literal> then all partitions must have OIDs; the parent's OID
@@ -487,7 +512,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -498,9 +523,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provide only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1651,6 +1690,16 @@ CREATE TABLE cities (
</programlisting></para>
<para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
+</programlisting></para>
+
+ <para>
Create partition of a range partitioned table:
<programlisting>
CREATE TABLE measurement_y2016m07
@@ -1703,6 +1752,19 @@ CREATE TABLE cities_ab_10000_to_100000
</programlisting></para>
<para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
+ <para>
Create a default partition:
<programlisting>
CREATE TABLE cities_partdef
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 73eff17..06d3dfc 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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);
@@ -147,6 +169,10 @@ static int32 partition_bound_cmp(PartitionKey key,
static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+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
@@ -172,6 +198,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;
@@ -253,7 +282,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;
@@ -482,6 +539,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));
@@ -615,8 +708,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];
@@ -653,49 +745,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;
+
+ /*
+ * If two hash partitioned tables have different greatest moduli or
+ * same moduli with different number of partitions, 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;
- for (j = 0; j < partnatts; j++)
+ /*
+ * We arrange the partitions in the ascending order of their modulus
+ * and remainders. Also every modulus is factor of next larger
+ * modulus. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -731,6 +871,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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);
@@ -1083,6 +1306,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);
@@ -1490,6 +1718,127 @@ make_partition_op_expr(PartitionKey key, int keynum,
}
/*
+ * 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 2, remainder 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 4, remainder 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns an implicit-AND list of expressions to use as a list partition's
@@ -2360,6 +2709,19 @@ 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 ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
if (isnull[0])
@@ -2473,6 +2835,38 @@ error_exit:
}
/*
+ * 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
*
* Compare two list partition bound datums
@@ -2658,6 +3052,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],
@@ -2842,3 +3245,104 @@ get_proposed_default_constraint(List *new_part_constraints)
return list_make1(defPartConstraint);
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 96354bd..35bf54c 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1bed14..ec6ac37 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4451,6 +4451,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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8b56b91..36c39fa 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2840,6 +2840,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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b83d919..02829e6 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3574,6 +3574,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index fbf8330..0f8674c 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2391,6 +2391,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index c303818..839276e 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -577,7 +577,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.
@@ -2636,8 +2637,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);
@@ -2675,6 +2729,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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 655da02..b315a69 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3309,6 +3309,11 @@ transformPartitionBound(ParseState *pstate, Relation parent,
if (spec->is_default)
{
+ if (strategy == PARTITION_STRATEGY_HASH)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("default hash partition is not supported")));
+
/*
* In case of the default partition, parser had no way to identify the
* partition strategy. Assign the parent's strategy to the default
@@ -3319,7 +3324,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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 0ea5078..5a9eaca 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8707,6 +8711,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e3780..c58d1e1 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d..b3e3799 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 454a940..e5b02ce 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index f73c6c6..fa4f83f 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5538,6 +5538,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index f3e4c69..028c531 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 0478a8a..124cbe4 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3474,6 +3475,59 @@ DETAIL: "part_5" is already a child of "list_parted2".
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
ERROR: circular inheritance not allowed
DETAIL: "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3485,12 +3539,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3587,6 +3646,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 58c755b..c389204 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (modu...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index 73a5600..d5332ad 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1..52842e3 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 37cca72..783229b 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2285,6 +2286,62 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2296,12 +2353,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2374,6 +2435,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index eeab5d9..3427ee5 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index a2948e4..f39d180 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1fec..9709424 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 17ba2bd..6d52f84 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.6.2
Hi Amul,
On 09/14/2017 04:58 AM, amul sul wrote:
On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen <jesper.pedersen@redhat.com
This patch needs a rebase.
Thanks for your note.
Attached is the patch rebased on the latest master head.
Also added error on creating default partition for the hash partitioned table,
and updated document & test script for the same.
Thanks !
When I do
CREATE TABLE mytab (
a integer NOT NULL,
b integer NOT NULL,
c integer,
d integer
) PARTITION BY HASH (b);
and create 64 partitions;
CREATE TABLE mytab_p00 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
REMAINDER 0);
...
CREATE TABLE mytab_p63 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
REMAINDER 63);
and associated indexes
CREATE INDEX idx_p00 ON mytab_p00 USING btree (b, a);
...
CREATE INDEX idx_p63 ON mytab_p63 USING btree (b, a);
Populate the database, and do ANALYZE.
Given
EXPLAIN (ANALYZE, VERBOSE, BUFFERS ON) SELECT a, b, c, d FROM mytab
WHERE b = 42
gives
Append
-> Index Scan using idx_p00 (cost rows=7) (actual rows=0)
...
-> Index Scan using idx_p63 (cost rows=7) (actual rows=0)
E.g. all partitions are being scanned. Of course one partition will
contain the rows I'm looking for.
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 14, 2017 at 11:39 AM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
When I do
CREATE TABLE mytab (
a integer NOT NULL,
b integer NOT NULL,
c integer,
d integer
) PARTITION BY HASH (b);and create 64 partitions;
CREATE TABLE mytab_p00 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
REMAINDER 0);
...
CREATE TABLE mytab_p63 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
REMAINDER 63);and associated indexes
CREATE INDEX idx_p00 ON mytab_p00 USING btree (b, a);
...
CREATE INDEX idx_p63 ON mytab_p63 USING btree (b, a);Populate the database, and do ANALYZE.
Given
EXPLAIN (ANALYZE, VERBOSE, BUFFERS ON) SELECT a, b, c, d FROM mytab WHERE b
= 42gives
Append
-> Index Scan using idx_p00 (cost rows=7) (actual rows=0)
...
-> Index Scan using idx_p63 (cost rows=7) (actual rows=0)E.g. all partitions are being scanned. Of course one partition will contain
the rows I'm looking for.
Yeah, we need Amit Langote's work in
/messages/by-id/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp
to land and this patch to be adapted to make use of it. I think
that's the major thing still standing in the way of this. Concerns
were also raised about not having a way to see the hash function, but
we fixed that in 81c5e46c490e2426db243eada186995da5bb0ba7 and
hopefully this patch has been updated to use a seed (I haven't looked
yet). And there was a concern about hash functions not being
portable, but the conclusion of that was basically that most people
think --load-via-partition-root will be a satisfactory workaround for
cases where that becomes a problem (cf. commit
23d7680d04b958de327be96ffdde8f024140d50e). So this is the major
remaining issue that I know about.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
On 09/14/2017 12:05 PM, Robert Haas wrote:
On Thu, Sep 14, 2017 at 11:39 AM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:When I do
CREATE TABLE mytab (
a integer NOT NULL,
b integer NOT NULL,
c integer,
d integer
) PARTITION BY HASH (b);and create 64 partitions;
CREATE TABLE mytab_p00 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
REMAINDER 0);
...
CREATE TABLE mytab_p63 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
REMAINDER 63);and associated indexes
CREATE INDEX idx_p00 ON mytab_p00 USING btree (b, a);
...
CREATE INDEX idx_p63 ON mytab_p63 USING btree (b, a);Populate the database, and do ANALYZE.
Given
EXPLAIN (ANALYZE, VERBOSE, BUFFERS ON) SELECT a, b, c, d FROM mytab WHERE b
= 42gives
Append
-> Index Scan using idx_p00 (cost rows=7) (actual rows=0)
...
-> Index Scan using idx_p63 (cost rows=7) (actual rows=0)E.g. all partitions are being scanned. Of course one partition will contain
the rows I'm looking for.Yeah, we need Amit Langote's work in
/messages/by-id/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp
to land and this patch to be adapted to make use of it. I think
that's the major thing still standing in the way of this. Concerns
were also raised about not having a way to see the hash function, but
we fixed that in 81c5e46c490e2426db243eada186995da5bb0ba7 and
hopefully this patch has been updated to use a seed (I haven't looked
yet). And there was a concern about hash functions not being
portable, but the conclusion of that was basically that most people
think --load-via-partition-root will be a satisfactory workaround for
cases where that becomes a problem (cf. commit
23d7680d04b958de327be96ffdde8f024140d50e). So this is the major
remaining issue that I know about.
Thanks for the information, Robert !
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Sep 11, 2017 at 07:43:29AM -0400, Robert Haas wrote:
On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:Rebased 0002 against this commit & renamed to 0001, PFA.
Given that we have default partition support now, I am wondering
whether hash partitioned tables also should have default
partitions. The way we have structured hash partitioning syntax,
there can be "holes" in partitions. Default partition would help
plug those holes.Yeah, I was thinking about that, too. On the one hand, it seems
like it's solving the problem the wrong way: if you've set up hash
partitioning properly, you shouldn't have any holes.
Should we be pointing the gun away from people's feet by making hash
partitions that cover the space automagically when the partitioning
scheme[1]For now, that's just the modulus, but the PoC included specifying hashing functions, so I assume other ways to specify the partitioning scheme could eventually be proposed. -- David Fetter <david(at)fetter(dot)org> http://fetter.org/ Phone: +1 415 235 3778 AIM: dfetter666 Yahoo!: dfetter Skype: davidfetter XMPP: david(dot)fetter(at)gmail(dot)com is specified? In other words, do we have a good reason to have
only some of the hash partitions so defined by default?
Best,
David.
[1]: For now, that's just the modulus, but the PoC included specifying hashing functions, so I assume other ways to specify the partitioning scheme could eventually be proposed. -- David Fetter <david(at)fetter(dot)org> http://fetter.org/ Phone: +1 415 235 3778 AIM: dfetter666 Yahoo!: dfetter Skype: davidfetter XMPP: david(dot)fetter(at)gmail(dot)com
hashing functions, so I assume other ways to specify the partitioning
scheme could eventually be proposed.
--
David Fetter <david(at)fetter(dot)org> http://fetter.org/
Phone: +1 415 235 3778 AIM: dfetter666 Yahoo!: dfetter
Skype: davidfetter XMPP: david(dot)fetter(at)gmail(dot)com
Remember to vote!
Consider donating to Postgres: http://www.postgresql.org/about/donate
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 14, 2017 at 12:54 PM, David Fetter <david@fetter.org> wrote:
Should we be pointing the gun away from people's feet by making hash
partitions that cover the space automagically when the partitioning
scheme[1] is specified? In other words, do we have a good reason to have
only some of the hash partitions so defined by default?
Sure, we can add some convenience syntax for that, but I'd like to get
the basic stuff working before doing that kind of polishing.
If nothing else, I assume Keith Fiske's pg_partman will provide a way
to magically DTRT about an hour after this goes in. But probably we
can do better in core easily enough.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 09/14/2017 12:56 PM, Robert Haas wrote:
On Thu, Sep 14, 2017 at 12:54 PM, David Fetter <david@fetter.org> wrote:
Should we be pointing the gun away from people's feet by making hash
partitions that cover the space automagically when the partitioning
scheme[1] is specified? In other words, do we have a good reason to have
only some of the hash partitions so defined by default?Sure, we can add some convenience syntax for that, but I'd like to get
the basic stuff working before doing that kind of polishing.If nothing else, I assume Keith Fiske's pg_partman will provide a way
to magically DTRT about an hour after this goes in. But probably we
can do better in core easily enough.
Yeah, it would be nice to have a syntax like
) PARTITION BY HASH (col) WITH (AUTO_CREATE = 64);
But then there also needs to be a way to create the 64 associated
indexes too for everything to be easy.
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 14, 2017 at 1:07 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
Yeah, it would be nice to have a syntax like
) PARTITION BY HASH (col) WITH (AUTO_CREATE = 64);
But then there also needs to be a way to create the 64 associated indexes
too for everything to be easy.
Well, for that, there's this proposal:
/messages/by-id/c8fe4f6b-ff46-aae0-89e3-e936a35f0cfd@postgrespro.ru
As several people have right pointed out, there's a lot of work to be
done on partitioning it to get it to where we want it to be. Even in
v10, it's got significant benefits, such as much faster bulk-loading,
but I don't hear anybody disputing the notion that a lot more work is
needed. The good news is that a lot of that work is already in
progress; the bad news is that a lot of that work is not done yet.
But I think that's OK. We can't solve every problem at once, and I
think we're moving things along here at a reasonably brisk pace. That
didn't stop me from complaining bitterly to someone just yesterday
that we aren't moving faster still, but unfortunately EnterpriseDB has
only been able to get 12 developers to do any work at all on
partitioning this release cycle, and 3 of those have so far helped
only with review and benchmarking. It's a pity we can't do more, but
considering how many community projects are 1-person efforts I think
it's pretty good.
To be clear, I know you're not (or at least I assume you're not)
trying to beat me up about this, just raising a concern, and I'm not
trying to beat you up either, just let you know that it is definitely
on the radar screen but not there yet.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 09/14/2017 01:52 PM, Robert Haas wrote:
On Thu, Sep 14, 2017 at 1:07 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:Yeah, it would be nice to have a syntax like
) PARTITION BY HASH (col) WITH (AUTO_CREATE = 64);
But then there also needs to be a way to create the 64 associated indexes
too for everything to be easy.Well, for that, there's this proposal:
/messages/by-id/c8fe4f6b-ff46-aae0-89e3-e936a35f0cfd@postgrespro.ru
As several people have right pointed out, there's a lot of work to be
done on partitioning it to get it to where we want it to be. Even in
v10, it's got significant benefits, such as much faster bulk-loading,
but I don't hear anybody disputing the notion that a lot more work is
needed. The good news is that a lot of that work is already in
progress; the bad news is that a lot of that work is not done yet.But I think that's OK. We can't solve every problem at once, and I
think we're moving things along here at a reasonably brisk pace. That
didn't stop me from complaining bitterly to someone just yesterday
that we aren't moving faster still, but unfortunately EnterpriseDB has
only been able to get 12 developers to do any work at all on
partitioning this release cycle, and 3 of those have so far helped
only with review and benchmarking. It's a pity we can't do more, but
considering how many community projects are 1-person efforts I think
it's pretty good.To be clear, I know you're not (or at least I assume you're not)
trying to beat me up about this, just raising a concern, and I'm not
trying to beat you up either, just let you know that it is definitely
on the radar screen but not there yet.
Definitely not a complain about the work being done.
I think the scope of Amul's and others work on hash partition support is
where it needs to be. Improvements can always follow in future release.
My point was that is easy to script the definition of the partitions and
their associated indexes, so it is more important to focus on the core
functionality with the developer / review resources available.
However, it is a little bit difficult to follow the dependencies between
different partition patches, so I may not always provide sane feedback,
as seen in [1]/messages/by-id/579077fd-8f07-aff7-39bc-b92c855cdb70@redhat.com.
[1]: /messages/by-id/579077fd-8f07-aff7-39bc-b92c855cdb70@redhat.com
/messages/by-id/579077fd-8f07-aff7-39bc-b92c855cdb70@redhat.com
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 14, 2017 at 2:05 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
However, it is a little bit difficult to follow the dependencies between
different partition patches, so I may not always provide sane feedback, as
seen in [1].[1]
/messages/by-id/579077fd-8f07-aff7-39bc-b92c855cdb70@redhat.com
Yeah, no issues. I knew about the dependency between those patches,
but I'm pretty sure there wasn't any terribly explicit discussion
about it, even if the issue probably came up parenthetically someplace
or other. Oops.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 14 September 2017 at 09:58, amul sul <sulamul@gmail.com> wrote:
On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:Hi Amul,
On 09/08/2017 08:40 AM, amul sul wrote:
Rebased 0002 against this commit & renamed to 0001, PFA.
This patch needs a rebase.
Thanks for your note.
Attached is the patch rebased on the latest master head.
Also added error on
creating
d
efault partition
for the hash partitioned table
,
and updated document &
test script for the same.
Sorry, but this needs another rebase as it's broken by commit
77b6b5e9ceca04dbd6f0f6cd3fc881519acc8714.
Thom
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Sep 15, 2017 at 4:30 AM, Thom Brown <thom@linux.com> wrote:
On 14 September 2017 at 09:58, amul sul <sulamul@gmail.com> wrote:
On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:Hi Amul,
On 09/08/2017 08:40 AM, amul sul wrote:
Rebased 0002 against this commit & renamed to 0001, PFA.
This patch needs a rebase.
Thanks for your note.
Attached is the patch rebased on the latest master head.
Also added error on
creating
d
efault partition
for the hash partitioned table
,
and updated document &
test script for the same.Sorry, but this needs another rebase as it's broken by commit
77b6b5e9ceca04dbd6f0f6cd3fc881519acc8714.
Attached rebased patch, thanks.
Regards,
Amul
Attachments:
0001-hash-partitioning_another_design-v20.patchapplication/octet-stream; name=0001-hash-partitioning_another_design-v20.patchDownload
From e1c793f8cbb4b69375fe99c682f250862433ec89 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 15 Sep 2017 11:52:51 +0530
Subject: [PATCH] hash-partitioning_another_design-v20
v20:
Rebased on lastest head(eaa4070).
v19:
Rebased on latest head.
Added error and test case for a default hash partition.
v18:
0001-Cleanup_v6.patch got committed(f0a0c17).
Rebased against patch against latest head.
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 590 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1188 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2..24b36ca 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying modulus and remainder for each
+ partition. Each partition holds rows for which the hash value of
+ partition keys when divided by specified modulus produces specified
+ remainder. For more clarification on modulus and remainder please refer
+ <xref linkend="sql-createtable-partition">.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385e..a6eefb8 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1421,6 +1421,13 @@ ALTER TABLE cities
</programlisting></para>
<para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (modulus 4, remainder 3);
+</programlisting></para>
+
+ <para>
Detach a partition from partitioned table:
<programlisting>
ALTER TABLE measurement
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 824253d..7e57e22 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -365,6 +367,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</para>
<para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the children of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 children, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
+ <para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
OIDS</literal> then all partitions must have OIDs; the parent's OID
@@ -487,7 +512,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -498,9 +523,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provide only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1651,6 +1690,16 @@ CREATE TABLE cities (
</programlisting></para>
<para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
+</programlisting></para>
+
+ <para>
Create partition of a range partitioned table:
<programlisting>
CREATE TABLE measurement_y2016m07
@@ -1703,6 +1752,19 @@ CREATE TABLE cities_ab_10000_to_100000
</programlisting></para>
<para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
+ <para>
Create a default partition:
<programlisting>
CREATE TABLE cities_partdef
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1ab6dba..5c11c7e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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 +171,10 @@ 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 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 +200,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 +284,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 +541,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 +710,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 +747,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;
+
+ /*
+ * If two hash partitioned tables have different greatest moduli or
+ * same moduli with different number of partitions, 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;
- for (j = 0; j < partnatts; j++)
+ /*
+ * We arrange the partitions in the ascending order of their modulus
+ * and remainders. Also every modulus is factor of next larger
+ * modulus. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -733,6 +873,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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);
@@ -1085,6 +1308,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);
@@ -1456,6 +1684,127 @@ make_partition_op_expr(PartitionKey key, int keynum,
}
/*
+ * 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 2, remainder 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 4, remainder 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (modulus 8, remainder 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns an implicit-AND list of expressions to use as a list partition's
@@ -2326,6 +2675,19 @@ 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 ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
if (isnull[0])
@@ -2439,6 +2801,38 @@ error_exit:
}
/*
+ * 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
*
* Compare two list partition bound datums
@@ -2624,6 +3018,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],
@@ -2808,3 +3211,104 @@ get_proposed_default_constraint(List *new_part_constraints)
return list_make1(defPartConstraint);
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 563bcda..2972a6e 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1bed14..ec6ac37 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4451,6 +4451,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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8b56b91..36c39fa 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2840,6 +2840,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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b83d919..02829e6 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3574,6 +3574,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index fbf8330..0f8674c 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2391,6 +2391,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index c303818..839276e 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -577,7 +577,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.
@@ -2636,8 +2637,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);
@@ -2675,6 +2729,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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 655da02..b315a69 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3309,6 +3309,11 @@ transformPartitionBound(ParseState *pstate, Relation parent,
if (spec->is_default)
{
+ if (strategy == PARTITION_STRATEGY_HASH)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("default hash partition is not supported")));
+
/*
* In case of the default partition, parser had no way to identify the
* partition strategy. Assign the parent's strategy to the default
@@ -3319,7 +3324,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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 0ea5078..5a9eaca 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8707,6 +8711,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e3780..c58d1e1 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d..b3e3799 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 454a940..e5b02ce 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index f73c6c6..fa4f83f 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5538,6 +5538,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index f3e4c69..028c531 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 0478a8a..124cbe4 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3474,6 +3475,59 @@ DETAIL: "part_5" is already a child of "list_parted2".
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
ERROR: circular inheritance not allowed
DETAIL: "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3485,12 +3539,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3587,6 +3646,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 58c755b..c389204 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (modu...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index 73a5600..d5332ad 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1..52842e3 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 37cca72..783229b 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2285,6 +2286,62 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2296,12 +2353,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2374,6 +2435,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index eeab5d9..3427ee5 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index a2948e4..f39d180 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1fec..9709424 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da..18d5641 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.6.2
On 09/15/2017 02:30 AM, amul sul wrote:
Attached rebased patch, thanks.
While reading through the patch I thought it would be better to keep
MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order
to highlight that these are "keywords" for hash partition.
Also updated some of the documentation.
V20 patch passes make check-world, and my testing (typical 64
partitions, and various ATTACH/DETACH scenarios).
Thanks for working on this !
Best regards,
Jesper
Attachments:
delta_v20_v1.patchtext/x-patch; name=delta_v20_v1.patchDownload
From 189a40a5ca6c7a1bc79b750cbc95584b3061fda5 Mon Sep 17 00:00:00 2001
From: jesperpedersen <jesper.pedersen@redhat.com>
Date: Mon, 18 Sep 2017 11:13:54 -0400
Subject: [PATCH] * Documentation updates * Use caps for MODULUS / REMAINDER
when CREATE TABLE is so too
---
doc/src/sgml/ddl.sgml | 10 ++++-----
doc/src/sgml/ref/alter_table.sgml | 2 +-
src/backend/catalog/partition.c | 8 +++----
src/test/regress/expected/alter_table.out | 20 +++++++++---------
src/test/regress/expected/create_table.out | 34 +++++++++++++++---------------
src/test/regress/sql/alter_table.sql | 20 +++++++++---------
src/test/regress/sql/create_table.sql | 28 ++++++++++++------------
7 files changed, 61 insertions(+), 61 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 24b36caad3..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2881,11 +2881,11 @@ VALUES ('Albany', NULL, NULL, 'NY');
<listitem>
<para>
- The table is partitioned by specifying modulus and remainder for each
- partition. Each partition holds rows for which the hash value of
- partition keys when divided by specified modulus produces specified
- remainder. For more clarification on modulus and remainder please refer
- <xref linkend="sql-createtable-partition">.
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index a6eefb8564..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1424,7 +1424,7 @@ ALTER TABLE cities
Attach a partition to hash partitioned table:
<programlisting>
ALTER TABLE orders
- ATTACH PARTITION orders_p4 FOR VALUES WITH (modulus 4, remainder 3);
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
</programlisting></para>
<para>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 5c11c7ecea..3696b9a711 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1694,13 +1694,13 @@ make_partition_op_expr(PartitionKey key, int keynum,
* CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
*
* CREATE TABLE p_p1 PARTITION OF simple_hash
- * FOR VALUES WITH (modulus 2, remainder 1);
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
* CREATE TABLE p_p2 PARTITION OF simple_hash
- * FOR VALUES WITH (modulus 4, remainder 2);
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
* CREATE TABLE p_p3 PARTITION OF simple_hash
- * FOR VALUES WITH (modulus 8, remainder 0);
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
* CREATE TABLE p_p4 PARTITION OF simple_hash
- * FOR VALUES WITH (modulus 8, remainder 4);
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
*
* This function will return one of the following in the form of an
* expression:
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 124cbe483c..304fb97291 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3490,22 +3490,22 @@ CREATE TABLE hash_parted (
a int,
b int
) PARTITION BY HASH (a custom_opclass);
-CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
CREATE TABLE fail_part (LIKE hpart_1);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
ERROR: partition "fail_part" would overlap partition "hpart_1"
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
ERROR: partition "fail_part" would overlap partition "hpart_1"
DROP TABLE fail_part;
-- check validation when attaching hash partitions
-- check that violating rows are correctly reported
CREATE TABLE hpart_2 (LIKE hash_parted);
INSERT INTO hpart_2 VALUES (3, 0);
-ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
ERROR: partition constraint is violated by some row
-- should be ok after deleting the bad row
DELETE FROM hpart_2;
-ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
-- check that leaf partitions are scanned when attaching a partitioned
-- table
CREATE TABLE hpart_5 (
@@ -3514,18 +3514,18 @@ CREATE TABLE hpart_5 (
-- check that violating rows are correctly reported
CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
-ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
ERROR: partition constraint is violated by some row
-- should be ok after deleting the bad row
DELETE FROM hpart_5_a;
-ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
-- check that the table being attach is with valid modulus and remainder value
CREATE TABLE fail_part(LIKE hash_parted);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
ERROR: modulus for hash partition must be a positive integer
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
ERROR: modulus for hash partition must be greater than remainder
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
ERROR: every hash partition modulus must be a factor of the next larger modulus
DROP TABLE fail_part;
--
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index a4127efddb..32ef71fc13 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -463,9 +463,9 @@ ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
-- trying to specify modulus and remainder for list partitioned table
-CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
ERROR: invalid bound specification for a list partition
-LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (modu...
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -510,9 +510,9 @@ ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
-- trying to specify modulus and remainder for range partitioned table
-CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
ERROR: invalid bound specification for a range partition
-LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
@@ -524,18 +524,18 @@ ERROR: TO must specify exactly one value per partitioning column
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
-- trying to specify modulus and remainder for range partitioned table
-CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
ERROR: invalid bound specification for a range partition
-LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (modu...
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
^
-- check partition bound syntax for the hash partition
CREATE TABLE hash_parted (
a int
) PARTITION BY HASH (a);
-CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
-CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
-CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
ERROR: every hash partition modulus must be a factor of the next larger modulus
-- trying to specify range for the hash partitioned table
CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -557,7 +557,7 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
-CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -661,18 +661,18 @@ ERROR: partition "fail_part" would overlap partition "part10"
CREATE TABLE hash_parted2 (
a varchar
) PARTITION BY HASH (a);
-CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
-CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
-CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
-CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
-- overlap with part_4
-CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
ERROR: partition "fail_part" would overlap partition "h2part_4"
-- modulus must be greater than zero
-CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
ERROR: modulus for hash partition must be a positive integer
-- remainder must be greater than or equal to zero and less than modulus
-CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 783229b942..7d9e8fa4a4 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2303,10 +2303,10 @@ CREATE TABLE hash_parted (
a int,
b int
) PARTITION BY HASH (a custom_opclass);
-CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 0);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
CREATE TABLE fail_part (LIKE hpart_1);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 4);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 0);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
DROP TABLE fail_part;
-- check validation when attaching hash partitions
@@ -2314,11 +2314,11 @@ DROP TABLE fail_part;
-- check that violating rows are correctly reported
CREATE TABLE hpart_2 (LIKE hash_parted);
INSERT INTO hpart_2 VALUES (3, 0);
-ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
-- should be ok after deleting the bad row
DELETE FROM hpart_2;
-ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
-- check that leaf partitions are scanned when attaching a partitioned
-- table
@@ -2329,17 +2329,17 @@ CREATE TABLE hpart_5 (
-- check that violating rows are correctly reported
CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
-ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
-- should be ok after deleting the bad row
DELETE FROM hpart_5_a;
-ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
-- check that the table being attach is with valid modulus and remainder value
CREATE TABLE fail_part(LIKE hash_parted);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
-ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
DROP TABLE fail_part;
--
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index 8c41016a9b..bb78df3247 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -442,7 +442,7 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
-- trying to specify modulus and remainder for list partitioned table
-CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -479,7 +479,7 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
-- trying to specify modulus and remainder for range partitioned table
-CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,16 +489,16 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
-- trying to specify modulus and remainder for range partitioned table
-CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (modulus 10, remainder 1);
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check partition bound syntax for the hash partition
CREATE TABLE hash_parted (
a int
) PARTITION BY HASH (a);
-CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10, remainder 1);
-CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (modulus 50, remainder 0);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
-CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (modulus 25, remainder 2);
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
-- trying to specify range for the hash partitioned table
CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
-- trying to specify list value for the hash partitioned table
@@ -514,7 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
-CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (modulus 2, remainder 1);
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -608,16 +608,16 @@ CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue)
CREATE TABLE hash_parted2 (
a varchar
) PARTITION BY HASH (a);
-CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 4, remainder 2);
-CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 0);
-CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 4);
-CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 5);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
-- overlap with part_4
-CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 2, remainder 1);
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
-- modulus must be greater than zero
-CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 0, remainder 1);
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
-- remainder must be greater than or equal to zero and less than modulus
-CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (modulus 8, remainder 8);
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
-- check schema propagation from parent
--
2.13.5
On Mon, Sep 18, 2017 at 8:55 PM, Jesper Pedersen <jesper.pedersen@redhat.com
wrote:
On 09/15/2017 02:30 AM, amul sul wrote:
Attached rebased patch, thanks.
While reading through the patch I thought it would be better to keep
MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
highlight that these are "keywords" for hash partition.Also updated some of the documentation.
Thanks a lot for the patch, included in the attached version.
V20 patch passes make check-world, and my testing (typical 64 partitions,
and various ATTACH/DETACH scenarios).
Nice, thanks again.
Regards,
Amul
Attachments:
0001-hash-partitioning_another_design-v21.patchapplication/octet-stream; name=0001-hash-partitioning_another_design-v21.patchDownload
From 08e6dee07189e4bb97ff1a73413bd14e7f0c87b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 15 Sep 2017 11:52:51 +0530
Subject: [PATCH] hash-partitioning_another_design-v21
v21:
Includes documentation update patch provided by Jesper Pedersen
in message-id 4b1d256e-bec4-e126-faae-81e9c45c13c1@redhat.com
v20:
Rebased on lastest head(eaa4070).
v19:
Rebased on latest head.
Added error and test case for a default hash partition.
v18:
0001-Cleanup_v6.patch got committed(f0a0c17).
Rebased against patch against latest head.
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
more fixes
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 590 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1188 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2150..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385ece7..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1420,6 +1420,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 1477288851..dce9686e5d 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -363,6 +365,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the children of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 children, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,7 +511,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -497,9 +522,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provide only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1647,6 +1686,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1750,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1ab6dba7ae..3696b9a711 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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 +171,10 @@ 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 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 +200,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 +284,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 +541,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 +710,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 +747,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;
+
+ /*
+ * If two hash partitioned tables have different greatest moduli or
+ * same moduli with different number of partitions, 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;
- for (j = 0; j < partnatts; j++)
+ /*
+ * We arrange the partitions in the ascending order of their modulus
+ * and remainders. Also every modulus is factor of next larger
+ * modulus. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -733,6 +873,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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);
@@ -1085,6 +1308,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);
@@ -1455,6 +1683,127 @@ 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2326,6 +2675,19 @@ 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 ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
if (isnull[0])
@@ -2438,6 +2800,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
*
@@ -2624,6 +3018,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],
@@ -2808,3 +3211,104 @@ get_proposed_default_constraint(List *new_part_constraints)
return list_make1(defPartConstraint);
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 563bcda30c..2972a6e8aa 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1bed14e2b..ec6ac37a88 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4451,6 +4451,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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8b56b9146a..36c39fa7da 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2840,6 +2840,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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b83d919e40..02829e66a3 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3574,6 +3574,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index fbf8330735..0f8674ca52 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2391,6 +2391,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index c303818c9b..839276e6de 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -577,7 +577,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.
@@ -2636,8 +2637,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);
@@ -2675,6 +2729,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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..8c9e8ad158 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("default hash partition is not supported")));
+
/*
* 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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 84759b6149..ee4369652d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8708,6 +8712,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..c58d1e17b7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 454a940a23..e5b02ce0f8 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..470b1f6482 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index f3e4c69753..028c531081 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 0478a8ac60..304fb97291 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3474,6 +3475,59 @@ DETAIL: "part_5" is already a child of "list_parted2".
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
ERROR: circular inheritance not allowed
DETAIL: "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3485,12 +3539,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3587,6 +3646,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..32ef71fc13 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..2294427305 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..52842e30b4 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 37cca72620..7d9e8fa4a4 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2285,6 +2286,62 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2296,12 +2353,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2374,6 +2435,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..bb78df3247 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..3a55f50c07 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..9709424186 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da2ee..18d56417bd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
On 09/27/2017 03:05 AM, amul sul wrote:
Attached rebased patch, thanks.
While reading through the patch I thought it would be better to keep
MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
highlight that these are "keywords" for hash partition.Also updated some of the documentation.
Thanks a lot for the patch, included in the attached version.
Thank you.
Based on [1]/messages/by-id/CA+TgmoYsw3pusDen4_A44c7od+bEAST0eYo+jODtyofR0W2soQ@mail.gmail.com I have moved the patch to "Ready for Committer".
[1]: /messages/by-id/CA+TgmoYsw3pusDen4_A44c7od+bEAST0eYo+jODtyofR0W2soQ@mail.gmail.com
/messages/by-id/CA+TgmoYsw3pusDen4_A44c7od+bEAST0eYo+jODtyofR0W2soQ@mail.gmail.com
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017/09/27 22:41, Jesper Pedersen wrote:
On 09/27/2017 03:05 AM, amul sul wrote:
Attached rebased patch, thanks.
While reading through the patch I thought it would be better to keep
MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
highlight that these are "keywords" for hash partition.Also updated some of the documentation.
Thanks a lot for the patch, included in the attached version.
Thank you.
Based on [1] I have moved the patch to "Ready for Committer".
Thanks a lot Amul for working on this. Like Jesper said, the patch looks
pretty good overall. I was looking at the latest version with intent to
study certain things about hash partitioning the way patch implements it,
during which I noticed some things.
+ The modulus must be a positive integer, and the remainder must a
must be a
+ suppose you have a hash-partitioned table with 8 children, each of
which
+ has modulus 8, but find it necessary to increase the number of
partitions
+ to 16.
Might it be a good idea to say 8 "partitions" instead of "children" in the
first sentence?
+ each modulus-8 partition until none remain. While this may still
involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
I read the paragraph that ends with the above text and started wondering
if the example to redistribute data in hash partitions by detaching and
attaching with new modulus/remainder could be illustrated with an example?
Maybe in the examples section of the ALTER TABLE page?
+ Since hash operator class provide only equality, not ordering,
collation
Either "Since hash operator classes provide" or "Since hash operator class
provides"
Other than the above points, patch looks good.
By the way, I noticed a couple of things about hash partition constraints:
1. In get_qual_for_hash(), using
get_fn_expr_rettype(&key->partsupfunc[i]), which returns InvalidOid for
the lack of fn_expr being set to non-NULL value, causes funcrettype of the
FuncExpr being generated for hashing partition key columns to be set to
InvalidOid, which I think is wrong. That is, the following if condition
in get_fn_expr_rettype() is always satisfied:
if (!flinfo || !flinfo->fn_expr)
return InvalidOid;
I think we could use get_func_rettype(&key->partsupfunc[i].fn_oid)
instead. Attached patch
hash-v21-set-funcexpr-funcrettype-correctly.patch, which applies on top
v21 of your patch.
2. It seems that the reason constraint exclusion doesn't work with hash
partitions as implemented by the patch is that predtest.c:
operator_predicate_proof() returns false even without looking into the
hash partition constraint, which is of the following form:
satisfies_hash_partition(<mod>, <rem>, <key1-exthash>,..)
beccause the above constraint expression doesn't translate into a a binary
opclause (an OpExpr), which operator_predicate_proof() knows how to work
with. So, false is returned at the beginning of that function by the
following code:
if (!is_opclause(predicate))
return false;
For example,
create table p (a int) partition by hash (a);
create table p0 partition of p for values with (modulus 4, remainder 0);
create table p1 partition of p for values with (modulus 4, remainder 1);
\d+ p0
<...>
Partition constraint: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
-- both p0 and p1 scanned
explain select * from p where satisfies_hash_partition(4, 0,
hashint4extended(a, '8816678312871386367'::bigint));
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..96.50 rows=1700 width=4)
-> Seq Scan on p0 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
-> Seq Scan on p1 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
(5 rows)
-- both p0 and p1 scanned
explain select * from p where satisfies_hash_partition(4, 1,
hashint4extended(a, '8816678312871386367'::bigint));
QUERY PLAN
----------------------------------------------------------------------------------------------------
Append (cost=0.00..96.50 rows=1700 width=4)
-> Seq Scan on p0 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
'8816678312871386367'::bigint))
-> Seq Scan on p1 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
'8816678312871386367'::bigint))
(5 rows)
I looked into how satisfies_hash_partition() works and came up with an
idea that I think will make constraint exclusion work. What if we emitted
the hash partition constraint in the following form instead:
hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
<mod>) = <rem>
With that form, constraint exclusion seems to work as illustrated below:
\d+ p0
<...>
Partition constraint:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)
-- note only p0 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0;
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------
Append (cost=0.00..61.00 rows=13 width=4)
-> Seq Scan on p0 (cost=0.00..61.00 rows=13 width=4)
Filter:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)
(3 rows)
-- note only p1 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 1;
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------
Append (cost=0.00..61.00 rows=13 width=4)
-> Seq Scan on p1 (cost=0.00..61.00 rows=13 width=4)
Filter:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 1)
(3 rows)
I tried to implement that in the attached
hash-v21-hash-part-constraint.patch, which applies on top v21 of your
patch (actually on top of
hash-v21-set-funcexpr-funcrettype-correctly.patch, which I think should be
applied anyway as it fixes a bug of the original patch).
What do you think? Eventually, the new partition-pruning method [1]https://commitfest.postgresql.org/14/1272/ will
make using constraint exclusion obsolete, but it might be a good idea to
have it working if we can.
Thanks,
Amit
Attachments:
hash-v21-hash-part-constraint-v1.patchtext/plain; charset=UTF-8; name=hash-v21-hash-part-constraint-v1.patchDownload
From 7773ba29d73e2e5cdb4fa7fee68c4598b70bf22f Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Thu, 28 Sep 2017 13:26:00 +0900
Subject: [PATCH 3/3] Make constraint exclusion work with hash partitions
---
src/backend/catalog/partition.c | 104 +++++++++++++++++++++++++++++-----------
src/include/catalog/pg_proc.h | 6 ++-
src/include/fmgr.h | 1 +
3 files changed, 82 insertions(+), 29 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index a085a7a0be..ba437c13c8 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -28,6 +28,7 @@
#include "catalog/pg_inherits.h"
#include "catalog/pg_inherits_fn.h"
#include "catalog/pg_opclass.h"
+#include "catalog/pg_operator.h"
#include "catalog/pg_partitioned_table.h"
#include "catalog/pg_type.h"
#include "commands/tablecmds.h"
@@ -174,7 +175,8 @@ static void get_partition_dispatch_recurse(Relation rel, Relation parent,
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);
+PG_FUNCTION_INFO_V1(hash_partition_modulus);
+PG_FUNCTION_INFO_V1(hash_partition_hash);
/*
* RelationBuildPartitionDesc
@@ -1705,14 +1707,14 @@ make_partition_op_expr(PartitionKey key, int keynum,
* This function will return one of the following in the form of an
* expression:
*
- * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
- * hash_fn_2_extended(b, HASH_SEED))
- * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
- * hash_fn_2_extended(b, HASH_SEED))
- * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
- * hash_fn_2_extended(b, HASH_SEED))
- * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
- * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p1: hash_partition_hash(hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED)) % 4 = 1
+ * for p_p2: hash_partition_hash(hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED)) % 4 = 2
+ * for p_p3: hash_partition_hash(hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED)) % 8 = 0
+ * for p_p4: hash_partition_hash(hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED)) % 8 = 4
*
* where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
* functions(extended version) for columns a and b respectively and HASH_SEED
@@ -1722,11 +1724,13 @@ static List *
get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
{
PartitionKey key = RelationGetPartitionKey(parent);
- FuncExpr *fexpr;
+ FuncExpr *hash_fexpr,
+ *mod_fexpr;
+ Expr *opexpr;
Node *modulusConst;
Node *remainderConst;
Node *hashSeedConst;
- List *args;
+ List *args = NIL;
ListCell *partexprs_item;
int i;
@@ -1758,13 +1762,13 @@ get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
false,
FLOAT8PASSBYVAL);
- args = list_make2(modulusConst, remainderConst);
partexprs_item = list_head(key->partexprs);
/* Add an argument for each key column. */
for (i = 0; i < key->partnatts; i++)
{
Node *keyCol;
+ FuncExpr *fexpr;
/* Left operand */
if (key->partattrs[i] != 0)
@@ -1794,14 +1798,26 @@ get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
args = lappend(args, fexpr);
}
- fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
- BOOLOID,
- args,
- InvalidOid,
- InvalidOid,
- COERCE_EXPLICIT_CALL);
+ hash_fexpr = makeFuncExpr(F_HASH_PARTITION_HASH,
+ INT8OID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+ mod_fexpr = makeFuncExpr(F_HASH_PARTITION_MODULUS,
+ INT4OID,
+ list_make2(hash_fexpr, modulusConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ /* Make an fexpr = true expression */
+ opexpr = make_opclause(Int4EqualOperator, BOOLOID, false,
+ (Expr *) mod_fexpr,
+ (Expr *) remainderConst,
+ InvalidOid, InvalidOid);
- return list_make1(fexpr);
+ return list_make1(opexpr);
}
/*
@@ -3273,7 +3289,43 @@ compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
}
/*
- * satisfies_hash_partition
+ * hash_partition_modulus
+ *
+ * This is a SQL-callable function for use in hash partition constraint that
+ * takes as arguments a 64-bit hash value and modulus and returns the
+ * remainder obtained by dividing the hash value with modulus.
+ */
+Datum
+hash_partition_modulus(PG_FUNCTION_ARGS)
+{
+ uint64 arg1 = PG_GETARG_UINT64(0);
+ int32 arg2 = PG_GETARG_INT32(1);
+
+ if (arg2 == 0)
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_DIVISION_BY_ZERO),
+ errmsg("division by zero")));
+ /* ensure compiler realizes we mustn't reach the division (gcc bug) */
+ PG_RETURN_NULL();
+ }
+
+ /*
+ * Some machines throw a floating-point exception for INT_MIN % -1, which
+ * is a bit silly since the correct answer is perfectly well-defined,
+ * namely zero. (It's not clear this ever happens when dealing with
+ * int16, but we might as well have the test for safety.)
+ */
+ if (arg2 == -1)
+ PG_RETURN_INT32(0);
+
+ /* No overflow is possible */
+
+ PG_RETURN_INT32(arg1 % arg2);
+}
+
+/*
+ * hash_partition_hash
*
* 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
@@ -3285,11 +3337,9 @@ compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
* See get_qual_for_hash() for usage.
*/
Datum
-satisfies_hash_partition(PG_FUNCTION_ARGS)
+hash_partition_hash(PG_FUNCTION_ARGS)
{
- int modulus = PG_GETARG_INT32(0);
- int remainder = PG_GETARG_INT32(1);
- short nkeys = PG_NARGS() - 2;
+ short nkeys = PG_NARGS();
int i;
Datum hash_array[PARTITION_MAX_KEYS];
bool isnull[PARTITION_MAX_KEYS];
@@ -3301,14 +3351,14 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
* Partition key attribute's hash values start from third argument of
* function.
*/
- isnull[i] = PG_ARGISNULL(i + 2);
+ isnull[i] = PG_ARGISNULL(i);
if (!isnull[i])
- hash_array[i] = PG_GETARG_DATUM(i + 2);
+ hash_array[i] = PG_GETARG_DATUM(i);
}
/* Form a single 64-bit hash value */
rowHash = mix_hash_value(nkeys, hash_array, isnull);
- PG_RETURN_BOOL(rowHash % modulus == remainder);
+ PG_RETURN_UINT64(rowHash);
}
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 470b1f6482..fa7e9a6f2b 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5523,8 +5523,10 @@ DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t
DESCR("list of files in the WAL directory");
/* hash partitioning constraint function */
-DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
-DESCR("hash partition CHECK constraint");
+DATA(insert OID = 315 ( hash_partition_modulus PGNSP PGUID 12 1 0 0 0 f f f f t f i s 2 0 23 "20 23" _null_ _null_ _null_ _null_ _null_ hash_partition_modulus _null_ _null_ _null_ ));
+DESCR("to use in hash partition CHECK constraint");
+DATA(insert OID = 5028 ( hash_partition_hash PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 1 0 20 "1007" _null_ _null_ _null_ _null_ _null_ hash_partition_hash _null_ _null_ _null_ ));
+DESCR("to use in hash partition CHECK constraint");
/*
* Symbolic values for provolatile column: these indicate whether the result
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index b604a5c162..7535f95382 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -245,6 +245,7 @@ extern struct varlena *pg_detoast_datum_packed(struct varlena *datum);
#define PG_GETARG_FLOAT4(n) DatumGetFloat4(PG_GETARG_DATUM(n))
#define PG_GETARG_FLOAT8(n) DatumGetFloat8(PG_GETARG_DATUM(n))
#define PG_GETARG_INT64(n) DatumGetInt64(PG_GETARG_DATUM(n))
+#define PG_GETARG_UINT64(n) DatumGetUInt64(PG_GETARG_DATUM(n))
/* use this if you want the raw, possibly-toasted input datum: */
#define PG_GETARG_RAW_VARLENA_P(n) ((struct varlena *) PG_GETARG_POINTER(n))
/* use this if you want the input datum de-toasted: */
--
2.11.0
hash-v21-set-funcexpr-funcrettype-correctly-v1.patchtext/plain; charset=UTF-8; name=hash-v21-set-funcexpr-funcrettype-correctly-v1.patchDownload
From 1f2bda192b3da7667fabaece2b7037e335ad6ec5 Mon Sep 17 00:00:00 2001
From: amit <amitlangote09@gmail.com>
Date: Thu, 28 Sep 2017 13:36:47 +0900
Subject: [PATCH 2/3] Fix get_qual_for_hash to set correct function return type
---
src/backend/catalog/partition.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 3696b9a711..a085a7a0be 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1785,7 +1785,7 @@ get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
/* Form hash_fn(keyCol) expression */
fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
- get_fn_expr_rettype(&key->partsupfunc[i]),
+ get_func_rettype(key->partsupfunc[i].fn_oid),
list_make2(keyCol, hashSeedConst),
InvalidOid,
InvalidOid,
--
2.11.0
On Thu, Sep 28, 2017 at 11:24 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
On 2017/09/27 22:41, Jesper Pedersen wrote:
On 09/27/2017 03:05 AM, amul sul wrote:
Attached rebased patch, thanks.
While reading through the patch I thought it would be better to keep
MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
highlight that these are "keywords" for hash partition.Also updated some of the documentation.
Thanks a lot for the patch, included in the attached version.
Thank you.
Based on [1] I have moved the patch to "Ready for Committer".
Thanks a lot Amul for working on this. Like Jesper said, the patch looks
pretty good overall. I was looking at the latest version with intent to
study certain things about hash partitioning the way patch implements it,
during which I noticed some things.
Thanks Amit for looking at the patch.
+ The modulus must be a positive integer, and the remainder must a
must be a
Fixed in the attached version.
+ suppose you have a hash-partitioned table with 8 children, each of which + has modulus 8, but find it necessary to increase the number of partitions + to 16.
Fixed in the attached version.
Might it be a good idea to say 8 "partitions" instead of "children" in the
first sentence?+ each modulus-8 partition until none remain. While this may still involve + a large amount of data movement at each step, it is still better than + having to create a whole new table and move all the data at once. + </para> +
Fixed in the attached version.
I read the paragraph that ends with the above text and started wondering
if the example to redistribute data in hash partitions by detaching and
attaching with new modulus/remainder could be illustrated with an example?
Maybe in the examples section of the ALTER TABLE page?
I think hint in the documentation is more than enough. There is N number of
ways of data redistribution, the document is not meant to explain all of those.
+ Since hash operator class provide only equality, not ordering,
collationEither "Since hash operator classes provide" or "Since hash operator class
provides"
Fixed in the attached version.
Other than the above points, patch looks good.
By the way, I noticed a couple of things about hash partition constraints:
1. In get_qual_for_hash(), using
get_fn_expr_rettype(&key->partsupfunc[i]), which returns InvalidOid for
the lack of fn_expr being set to non-NULL value, causes funcrettype of the
FuncExpr being generated for hashing partition key columns to be set to
InvalidOid, which I think is wrong. That is, the following if condition
in get_fn_expr_rettype() is always satisfied:if (!flinfo || !flinfo->fn_expr)
return InvalidOid;I think we could use get_func_rettype(&key->partsupfunc[i].fn_oid)
instead. Attached patch
hash-v21-set-funcexpr-funcrettype-correctly.patch, which applies on top
v21 of your patch.
Thanks for the patch, included in the attached version.
2. It seems that the reason constraint exclusion doesn't work with hash
partitions as implemented by the patch is that predtest.c:
operator_predicate_proof() returns false even without looking into the
hash partition constraint, which is of the following form:satisfies_hash_partition(<mod>, <rem>, <key1-exthash>,..)
beccause the above constraint expression doesn't translate into a a binary
opclause (an OpExpr), which operator_predicate_proof() knows how to work
with. So, false is returned at the beginning of that function by the
following code:if (!is_opclause(predicate))
return false;For example,
create table p (a int) partition by hash (a);
create table p0 partition of p for values with (modulus 4, remainder 0);
create table p1 partition of p for values with (modulus 4, remainder 1);
\d+ p0
<...>
Partition constraint: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))-- both p0 and p1 scanned
explain select * from p where satisfies_hash_partition(4, 0,
hashint4extended(a, '8816678312871386367'::bigint));
QUERY PLAN----------------------------------------------------------------------------------------------------
Append (cost=0.00..96.50 rows=1700 width=4)
-> Seq Scan on p0 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
-> Seq Scan on p1 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
(5 rows)-- both p0 and p1 scanned
explain select * from p where satisfies_hash_partition(4, 1,
hashint4extended(a, '8816678312871386367'::bigint));
QUERY PLAN----------------------------------------------------------------------------------------------------
Append (cost=0.00..96.50 rows=1700 width=4)
-> Seq Scan on p0 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
'8816678312871386367'::bigint))
-> Seq Scan on p1 (cost=0.00..48.25 rows=850 width=4)
Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
'8816678312871386367'::bigint))
(5 rows)I looked into how satisfies_hash_partition() works and came up with an
idea that I think will make constraint exclusion work. What if we emitted
the hash partition constraint in the following form instead:hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
<mod>) = <rem>With that form, constraint exclusion seems to work as illustrated below:
\d+ p0
<...>
Partition constraint:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)-- note only p0 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0;
QUERY PLAN--------------------------------------------------------------------------------------------------------------------------
Append (cost=0.00..61.00 rows=13 width=4)
-> Seq Scan on p0 (cost=0.00..61.00 rows=13 width=4)
Filter:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)
(3 rows)-- note only p1 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 1;
QUERY PLAN--------------------------------------------------------------------------------------------------------------------------
Append (cost=0.00..61.00 rows=13 width=4)
-> Seq Scan on p1 (cost=0.00..61.00 rows=13 width=4)
Filter:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 1)
(3 rows)I tried to implement that in the attached
hash-v21-hash-part-constraint.patch, which applies on top v21 of your
patch (actually on top of
hash-v21-set-funcexpr-funcrettype-correctly.patch, which I think should be
applied anyway as it fixes a bug of the original patch).What do you think? Eventually, the new partition-pruning method [1] will
make using constraint exclusion obsolete, but it might be a good idea to
have it working if we can.
It does not really do the partition pruning via constraint exclusion and I don't
think anyone is going to use the remainder in the where condition to fetch
data and hash partitioning is not meant for that.
But I am sure that we could solve this problem using your and Beena's work
toward faster partition pruning[1] and Runtime Partition Pruning[2].
Will think on this changes if it is required for the pruning feature.
Regards,
Amul
1] /messages/by-id/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp
2] /messages/by-id/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
Attachments:
0001-hash-partitioning_another_design-v22.patchapplication/octet-stream; name=0001-hash-partitioning_another_design-v22.patchDownload
From 0f8ad84f01148059c9c1adc7046fc15f41ddf7b4 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 15 Sep 2017 11:52:51 +0530
Subject: [PATCH] hash-partitioning_another_design-v22
v22:
Some fixes suggested by Amit Langote in message-id
cdd10cdc-0c92-8294-db56-3e89b65604b6@lab.ntt.co.jp
v21:
Includes documentation update patch provided by Jesper Pedersen
in message-id 4b1d256e-bec4-e126-faae-81e9c45c13c1@redhat.com
v20:
Rebased on lastest head(eaa4070).
v19:
Rebased on latest head.
Added error and test case for a default hash partition.
v18:
0001-Cleanup_v6.patch got committed(f0a0c17).
Rebased against patch against latest head.
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
more fixes
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 590 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1188 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2150..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385ece7..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1420,6 +1420,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 1477288851..43baff40e5 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -363,6 +365,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,7 +511,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -497,9 +522,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provides only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1647,6 +1686,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1750,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1ab6dba7ae..a085a7a0be 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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 +171,10 @@ 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 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 +200,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 +284,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 +541,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 +710,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 +747,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;
+
+ /*
+ * If two hash partitioned tables have different greatest moduli or
+ * same moduli with different number of partitions, 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;
- for (j = 0; j < partnatts; j++)
+ /*
+ * We arrange the partitions in the ascending order of their modulus
+ * and remainders. Also every modulus is factor of next larger
+ * modulus. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -733,6 +873,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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);
@@ -1085,6 +1308,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);
@@ -1455,6 +1683,127 @@ 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2326,6 +2675,19 @@ 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 ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
if (isnull[0])
@@ -2438,6 +2800,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
*
@@ -2624,6 +3018,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],
@@ -2808,3 +3211,104 @@ get_proposed_default_constraint(List *new_part_constraints)
return list_make1(defPartConstraint);
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 563bcda30c..2972a6e8aa 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1bed14e2b..ec6ac37a88 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4451,6 +4451,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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8b56b9146a..36c39fa7da 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2840,6 +2840,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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b83d919e40..02829e66a3 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3574,6 +3574,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index fbf8330735..0f8674ca52 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2391,6 +2391,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index c303818c9b..839276e6de 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -577,7 +577,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.
@@ -2636,8 +2637,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);
@@ -2675,6 +2729,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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..8c9e8ad158 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("default hash partition is not supported")));
+
/*
* 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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 84759b6149..ee4369652d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8708,6 +8712,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..c58d1e17b7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 454a940a23..e5b02ce0f8 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..470b1f6482 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index f3e4c69753..028c531081 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 0478a8ac60..304fb97291 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3474,6 +3475,59 @@ DETAIL: "part_5" is already a child of "list_parted2".
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
ERROR: circular inheritance not allowed
DETAIL: "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3485,12 +3539,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3587,6 +3646,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..32ef71fc13 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..2294427305 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..52842e30b4 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 37cca72620..7d9e8fa4a4 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2285,6 +2286,62 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2296,12 +2353,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2374,6 +2435,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..bb78df3247 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..3a55f50c07 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..9709424186 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da2ee..18d56417bd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
On Thu, Sep 28, 2017 at 1:54 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
I looked into how satisfies_hash_partition() works and came up with an
idea that I think will make constraint exclusion work. What if we emitted
the hash partition constraint in the following form instead:hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
<mod>) = <rem>With that form, constraint exclusion seems to work as illustrated below:
\d+ p0
<...>
Partition constraint:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)-- note only p0 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0;
What we actually want constraint exclusion to cover is SELECT * FROM p
WHERE a = 525600;
As Amul says, nobody's going to enter a query in the form you have it
here. Life is too short to take time to put queries into bizarre
forms.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi Amul,
On 09/28/2017 05:56 AM, amul sul wrote:
It does not really do the partition pruning via constraint exclusion and I don't
think anyone is going to use the remainder in the where condition to fetch
data and hash partitioning is not meant for that.But I am sure that we could solve this problem using your and Beena's work
toward faster partition pruning[1] and Runtime Partition Pruning[2].Will think on this changes if it is required for the pruning feature.
Could you rebase on latest master ?
Best regards,
Jesper
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Oct 6, 2017 at 5:35 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
Hi Amul,
Could you rebase on latest master ?
Sure will post that soon, but before that, I need to test hash partitioning
with recent partition-wise join commit (f49842d1ee), thanks.
Regards,
Amul
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Oct 7, 2017 at 5:22 PM, amul sul <sulamul@gmail.com> wrote:
On Fri, Oct 6, 2017 at 5:35 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:Hi Amul,
Could you rebase on latest master ?
Sure will post that soon, but before that, I need to test hash partitioning
with recent partition-wise join commit (f49842d1ee), thanks.
Updated patch attached.
0001 is the rebased of the previous patch, no new change.
0002 few changes in partition-wise join code to support
hash-partitioned table as well & regression tests.
Thanks & Regards,
Amul
Attachments:
0001-hash-partitioning_another_design-v23.patchapplication/octet-stream; name=0001-hash-partitioning_another_design-v23.patchDownload
From 81c98af12afa40703d7e86cdb9ecfa24623eb81b Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 9 Oct 2017 10:40:51 +0530
Subject: [PATCH 1/2] hash-partitioning_another_design-v23
v23:
Rebased on a latest master head.
v22:
Some fixes suggested by Amit Langote in message-id
cdd10cdc-0c92-8294-db56-3e89b65604b6@lab.ntt.co.jp
v21:
Includes documentation update patch provided by Jesper Pedersen
in message-id 4b1d256e-bec4-e126-faae-81e9c45c13c1@redhat.com
v20:
Rebased on lastest head(eaa4070).
v19:
Rebased on latest head.
Added error and test case for a default hash partition.
v18:
0001-Cleanup_v6.patch got committed(f0a0c17).
Rebased against patch against latest head.
v17:
Updated to work with extended hash function
commit # 81c5e46c490e2426db243eada186995da5bb0ba7
v16:
Rebased against latest master head(#efd7f8e).
v15:
Changes w.r.t Dilip Kumar's review comment in message-id:
CAFiTN-sXBP4V2AC3p4dfnUpOzQDDhe%3D6QS-yMqeYuz%2BfxKMHaQ%40mail.gmail.com
v14:
Changes w.r.t Yugo Nagata-san's review comment in message-id:
20170623134115.86f01d0c.nagata%40sraoss.co.jp
v13:
Couple of cosmetics fixes and Changes w.r.t Dilip's
review comments in message-id :
CAFiTN-tYoW9s0pL6cYkhGoniMVZi8%3DvHD0Q_KYE6xDcKN5SH7g%40mail.gmail.com
v12:
document update
v11:
Changes w.r.t Robert's review comments in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
more fixes
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 590 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1188 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2150..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385ece7..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1420,6 +1420,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 1477288851..43baff40e5 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -363,6 +365,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,7 +511,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -497,9 +522,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provides only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1647,6 +1686,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1750,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index ebda85e4ef..0818a7a7bc 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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 +171,10 @@ 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 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 +200,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 +284,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 +541,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 +710,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 +747,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 or
+ * same moduli with different number of partitions, 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. This means that the index of a given partition is same as
+ * the 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 = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+ 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -801,6 +941,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 = DatumGetInt32(datums[ndatums - 1][0]);
+ 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 +1394,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 +1769,127 @@ 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2412,6 +2761,19 @@ 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 ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int greatest_modulus = DatumGetInt32(datum);
+ 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 +2886,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 +3104,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 +3297,104 @@ get_proposed_default_constraint(List *new_part_constraints)
return list_make1(defPartConstraint);
}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 2d4dcd7556..a60f4a29ff 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca909..cadd253ef1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 7a700018e7..2866fd7b4a 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 2532edc94a..90995a118f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3573,6 +3573,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 07ba69178c..1331939479 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2390,6 +2390,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4c83a63f7d..92f5b86418 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..8c9e8ad158 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("default hash partition is not supported")));
+
/*
* 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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 84759b6149..ee4369652d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8708,6 +8712,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..c58d1e17b7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac0239d..de7919af45 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..470b1f6482 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 50eec730b3..bde2a020e8 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index dbe438dcd4..13b049009b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3494,6 +3495,59 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: partition constraint for table "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3505,12 +3559,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3607,6 +3666,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..32ef71fc13 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..2294427305 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..52842e30b4 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 0c8ae2ab97..972f5aae35 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2304,6 +2305,62 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2315,12 +2372,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2393,6 +2454,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..bb78df3247 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..3a55f50c07 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..9709424186 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da2ee..18d56417bd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
0002-Enable-partition-wise-join-support-v1.patchapplication/octet-stream; name=0002-Enable-partition-wise-join-support-v1.patchDownload
From b6b4694d9d3adaebeaefec8269ac5ce88a5e0168 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 9 Oct 2017 16:32:06 +0530
Subject: [PATCH 2/2] Enable partition wise join support v1
v1:
-Few changes to enable partition wise join support for
the hash partitioned table.
-Added regression tests.
---
src/backend/catalog/partition.c | 44 ++++++++++++---
src/backend/optimizer/path/joinrels.c | 10 +++-
src/test/regress/expected/partition_join.out | 81 ++++++++++++++++++++++++++++
src/test/regress/sql/partition_join.sql | 32 +++++++++++
4 files changed, 157 insertions(+), 10 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 0818a7a7bc..315755e32b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -861,11 +861,31 @@ 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;
+ switch (key->strategy)
+ {
+ case PARTITION_STRATEGY_HASH:
+ /*
+ * Indexes array is same as the greatest modulus.
+ * See partition_bounds_equal() for more explanation.
+ */
+ num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]);
+ break;
+
+ case PARTITION_STRATEGY_LIST:
+ /* List partitioned tables have only a single partition key. */
+ Assert(partnatts == 1);
+ num_indexes = ndatums;
+ break;
+
+ case PARTITION_STRATEGY_RANGE:
+ /* Range partitioned table has an extra index. */
+ num_indexes = ndatums + 1;
+ break;
- /* List partitioned tables have only a single partition key. */
- Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+ default:
+ elog(ERROR, "unexpected partition strategy: %d",
+ (int) key->strategy);
+ }
dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
@@ -888,15 +908,23 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
- dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
- 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 dim = hash_part? 2 : partnatts;
+
+ dest->datums[i] = (Datum *) palloc(sizeof(Datum) * dim);
+
+ for (j = 0; j < dim; 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]);
+ hash_part? true : key->parttypbyval[j],
+ key->parttyplen[j]);
}
}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c52de..5df861f9cf 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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. */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 234b8b5381..1c8cdb34fb 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1256,6 +1256,87 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
One-Time Filter: false
(14 rows)
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ QUERY PLAN
+--------------------------------------------------------------------------------------
+ Sort
+ Sort Key: t1.c, t3.c
+ -> HashAggregate
+ Group Key: t1.c, t2.c, t3.c
+ -> Result
+ -> Append
+ -> Hash Join
+ Hash Cond: (t1.c = t2.c)
+ -> Seq Scan on pht1_p1 t1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+ -> Seq Scan on pht2_p1 t2
+ -> Hash
+ -> Seq Scan on pht1_e_p1 t3
+ -> Hash Join
+ Hash Cond: (t1_1.c = t2_1.c)
+ -> Seq Scan on pht1_p2 t1_1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+ -> Seq Scan on pht2_p2 t2_1
+ -> Hash
+ -> Seq Scan on pht1_e_p2 t3_1
+ -> Hash Join
+ Hash Cond: (t1_2.c = t2_2.c)
+ -> Seq Scan on pht1_p3 t1_2
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+ -> Seq Scan on pht2_p3 t2_2
+ -> Hash
+ -> Seq Scan on pht1_e_p3 t3_2
+(33 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ avg | avg | avg | c | c | c
+----------------------+----------------------+-----------------------+------+------+-------
+ 24.0000000000000000 | 24.0000000000000000 | 48.0000000000000000 | 0000 | 0000 | A0000
+ 74.0000000000000000 | 75.0000000000000000 | 148.0000000000000000 | 0001 | 0001 | A0001
+ 124.0000000000000000 | 124.5000000000000000 | 248.0000000000000000 | 0002 | 0002 | A0002
+ 174.0000000000000000 | 174.0000000000000000 | 348.0000000000000000 | 0003 | 0003 | A0003
+ 224.0000000000000000 | 225.0000000000000000 | 448.0000000000000000 | 0004 | 0004 | A0004
+ 274.0000000000000000 | 274.5000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
+ 324.0000000000000000 | 324.0000000000000000 | 648.0000000000000000 | 0006 | 0006 | A0006
+ 374.0000000000000000 | 375.0000000000000000 | 748.0000000000000000 | 0007 | 0007 | A0007
+ 424.0000000000000000 | 424.5000000000000000 | 848.0000000000000000 | 0008 | 0008 | A0008
+ 474.0000000000000000 | 474.0000000000000000 | 948.0000000000000000 | 0009 | 0009 | A0009
+ 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
+ 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
+(12 rows)
+
--
-- multiple levels of partitioning
--
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index ca525d9941..2316bbdcb8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -229,6 +229,38 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+
--
-- multiple levels of partitioning
--
--
2.14.1
On Mon, Oct 9, 2017 at 4:44 PM, amul sul <sulamul@gmail.com> wrote:
0002 few changes in partition-wise join code to support
hash-partitioned table as well & regression tests.
+ switch (key->strategy)
+ {
+ case PARTITION_STRATEGY_HASH:
+ /*
+ * Indexes array is same as the greatest modulus.
+ * See partition_bounds_equal() for more explanation.
+ */
+ num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]);
+ break;
This logic is duplicated at multiple places. I think it's time we consolidate
these changes in a function/macro and call it from the places where we have to
calculate number of indexes based on the information in partition descriptor.
Refactoring existing code might be a separate patch and then add hash
partitioning case in hash partitioning patch.
+ int dim = hash_part? 2 : partnatts;
Call the variable as natts_per_datum or just natts?
+ hash_part? true : key->parttypbyval[j],
+ key->parttyplen[j]);
parttyplen is the length of partition key attribute, whereas what you want here
is the length of type of modulus and remainder. Is that correct? Probably we
need some special handling wherever parttyplen and parttypbyval is used e.g. in
call to partition_bounds_equal() from build_joinrel_partition_info().
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Oct 9, 2017 at 5:51 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Mon, Oct 9, 2017 at 4:44 PM, amul sul <sulamul@gmail.com> wrote:
Thanks Ashutosh for your review, please find my comment inline.
0002 few changes in partition-wise join code to support
hash-partitioned table as well & regression tests.+ switch (key->strategy) + { + case PARTITION_STRATEGY_HASH: + /* + * Indexes array is same as the greatest modulus. + * See partition_bounds_equal() for more explanation. + */ + num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]); + break; This logic is duplicated at multiple places. I think it's time we consolidate these changes in a function/macro and call it from the places where we have to calculate number of indexes based on the information in partition descriptor. Refactoring existing code might be a separate patch and then add hash partitioning case in hash partitioning patch.
Make sense, added get_partition_bound_num_indexes() to get number of index
elements in 0001 & get_greatest_modulus() as name suggested to get the greatest
modulus of the hash partition bound in 0002.
+ int dim = hash_part? 2 : partnatts;
Call the variable as natts_per_datum or just natts?
natts represents the number of attributes, but for the hash partition bound we
are not dealing with the attribute so that I have used short-form of dimension,
thoughts?
+ hash_part? true : key->parttypbyval[j], + key->parttyplen[j]); parttyplen is the length of partition key attribute, whereas what you want here is the length of type of modulus and remainder. Is that correct? Probably we need some special handling wherever parttyplen and parttypbyval is used e.g. in call to partition_bounds_equal() from build_joinrel_partition_info().
Unless I am missing something, I don't think we should worry about parttyplen
because in the datumCopy() when the datatype is pass-by-value then typelen
is ignored.
Regards,
Amul
Attachments:
0001-partition_bounds_copy-code-refactoring-v1.patchapplication/octet-stream; name=0001-partition_bounds_copy-code-refactoring-v1.patchDownload
From 64494e6be2d5e365e2161684791ae291a2751fcc Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:36:24 +0530
Subject: [PATCH 1/3] partition_bounds_copy code refactoring v1
---
src/backend/catalog/partition.c | 35 +++++++++++++++++++++++++++++++++--
1 file changed, 33 insertions(+), 2 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index ebda85e4ef..b82ceb903d 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -149,6 +149,7 @@ 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);
/*
* RelationBuildPartitionDesc
@@ -721,8 +722,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);
@@ -2894,3 +2894,34 @@ 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_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;
+}
--
2.14.1
0002-hash-partitioning_another_design-v24.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v24.patchDownload
From 4dd2fbf95b70f84897c50638798028f1ddffc011 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:50:49 +0530
Subject: [PATCH 2/3] hash-partitioning_another_design-v24
v24:
- Added get_greatest_modulus() to as per Ashutosh's suggestion[1].
1] https://postgr.es/m/CAFjFpRexH08-rb2LzodCsEeVTbxRQiNuYgh7XfGJ1Gd4zg+AaQ@mail.gmail.com
v23 - v11 history:
Refer 0001-v23 patch @
https://postgr.es/m/CAAJ_b94A6NCGVkdQxuhLR8x0=20vaD9EH8mQ0Tj8X3M06Tp-Ow@mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 614 +++++++++++++++++++++++++++--
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 +++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1212 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2150..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385ece7..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1420,6 +1420,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 1477288851..43baff40e5 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -363,6 +365,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,7 +511,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -497,9 +522,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provides only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1647,6 +1686,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1750,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b82ceb903d..1b894bd1be 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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);
@@ -150,6 +172,11 @@ static int partition_bound_bsearch(PartitionKey key,
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
@@ -175,6 +202,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;
@@ -256,7 +286,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;
@@ -485,6 +543,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));
@@ -618,8 +712,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];
@@ -656,49 +749,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 or
+ * same moduli with different number of partitions, 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. This means that the index of a given partition is same as
+ * the 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -801,6 +942,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 +1395,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 +1770,127 @@ 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2412,6 +2762,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 +2885,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 +3103,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],
@@ -2909,6 +3311,14 @@ get_partition_bound_num_indexes(PartitionBoundInfo 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;
@@ -2925,3 +3335,121 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
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]);
+}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 2d4dcd7556..a60f4a29ff 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca909..cadd253ef1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 7a700018e7..2866fd7b4a 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 2532edc94a..90995a118f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3573,6 +3573,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 07ba69178c..1331939479 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2390,6 +2390,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4c83a63f7d..92f5b86418 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..8c9e8ad158 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("default hash partition is not supported")));
+
/*
* 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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 84759b6149..ee4369652d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8708,6 +8712,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..c58d1e17b7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac0239d..de7919af45 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..470b1f6482 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 50eec730b3..bde2a020e8 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index dbe438dcd4..13b049009b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3494,6 +3495,59 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: partition constraint for table "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3505,12 +3559,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3607,6 +3666,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..32ef71fc13 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..2294427305 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..52842e30b4 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 0c8ae2ab97..972f5aae35 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2304,6 +2305,62 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2315,12 +2372,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2393,6 +2454,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..bb78df3247 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..3a55f50c07 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..9709424186 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da2ee..18d56417bd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
0003-Enable-partition-wise-join-support-v2.patchapplication/octet-stream; name=0003-Enable-partition-wise-join-support-v2.patchDownload
From 8634a6a99398cc8572e7848aeb956a4464312d76 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 15:08:34 +0530
Subject: [PATCH 3/3] Enable partition wise join support v2
v2:
Moved switch case from partition_bounds_copy() to 0001 patch.
v1:
-Few changes to enable partition wise join support for
the hash partitioned table.
-Added regression tests.
---
src/backend/catalog/partition.c | 16 ++++--
src/backend/optimizer/path/joinrels.c | 10 +++-
src/test/regress/expected/partition_join.out | 81 ++++++++++++++++++++++++++++
src/test/regress/sql/partition_join.sql | 32 +++++++++++
4 files changed, 133 insertions(+), 6 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1b894bd1be..61de13fb07 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -889,15 +889,23 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
- dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
- 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 dim = hash_part? 2 : partnatts;
+
+ dest->datums[i] = (Datum *) palloc(sizeof(Datum) * dim);
+
+ for (j = 0; j < dim; 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]);
+ hash_part? true : key->parttypbyval[j],
+ key->parttyplen[j]);
}
}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c52de..5df861f9cf 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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. */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 234b8b5381..1c8cdb34fb 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1256,6 +1256,87 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
One-Time Filter: false
(14 rows)
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ QUERY PLAN
+--------------------------------------------------------------------------------------
+ Sort
+ Sort Key: t1.c, t3.c
+ -> HashAggregate
+ Group Key: t1.c, t2.c, t3.c
+ -> Result
+ -> Append
+ -> Hash Join
+ Hash Cond: (t1.c = t2.c)
+ -> Seq Scan on pht1_p1 t1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+ -> Seq Scan on pht2_p1 t2
+ -> Hash
+ -> Seq Scan on pht1_e_p1 t3
+ -> Hash Join
+ Hash Cond: (t1_1.c = t2_1.c)
+ -> Seq Scan on pht1_p2 t1_1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+ -> Seq Scan on pht2_p2 t2_1
+ -> Hash
+ -> Seq Scan on pht1_e_p2 t3_1
+ -> Hash Join
+ Hash Cond: (t1_2.c = t2_2.c)
+ -> Seq Scan on pht1_p3 t1_2
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+ -> Seq Scan on pht2_p3 t2_2
+ -> Hash
+ -> Seq Scan on pht1_e_p3 t3_2
+(33 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ avg | avg | avg | c | c | c
+----------------------+----------------------+-----------------------+------+------+-------
+ 24.0000000000000000 | 24.0000000000000000 | 48.0000000000000000 | 0000 | 0000 | A0000
+ 74.0000000000000000 | 75.0000000000000000 | 148.0000000000000000 | 0001 | 0001 | A0001
+ 124.0000000000000000 | 124.5000000000000000 | 248.0000000000000000 | 0002 | 0002 | A0002
+ 174.0000000000000000 | 174.0000000000000000 | 348.0000000000000000 | 0003 | 0003 | A0003
+ 224.0000000000000000 | 225.0000000000000000 | 448.0000000000000000 | 0004 | 0004 | A0004
+ 274.0000000000000000 | 274.5000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
+ 324.0000000000000000 | 324.0000000000000000 | 648.0000000000000000 | 0006 | 0006 | A0006
+ 374.0000000000000000 | 375.0000000000000000 | 748.0000000000000000 | 0007 | 0007 | A0007
+ 424.0000000000000000 | 424.5000000000000000 | 848.0000000000000000 | 0008 | 0008 | A0008
+ 474.0000000000000000 | 474.0000000000000000 | 948.0000000000000000 | 0009 | 0009 | A0009
+ 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
+ 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
+(12 rows)
+
--
-- multiple levels of partitioning
--
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index ca525d9941..2316bbdcb8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -229,6 +229,38 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+
--
-- multiple levels of partitioning
--
--
2.14.1
On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
On Mon, Oct 9, 2017 at 5:51 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:On Mon, Oct 9, 2017 at 4:44 PM, amul sul <sulamul@gmail.com> wrote:
Thanks Ashutosh for your review, please find my comment inline.
0002 few changes in partition-wise join code to support
hash-partitioned table as well & regression tests.+ switch (key->strategy) + { + case PARTITION_STRATEGY_HASH: + /* + * Indexes array is same as the greatest modulus. + * See partition_bounds_equal() for more explanation. + */ + num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]); + break; This logic is duplicated at multiple places. I think it's time we consolidate these changes in a function/macro and call it from the places where we have to calculate number of indexes based on the information in partition descriptor. Refactoring existing code might be a separate patch and then add hash partitioning case in hash partitioning patch.Make sense, added get_partition_bound_num_indexes() to get number of index
elements in 0001 & get_greatest_modulus() as name suggested to get the greatest
modulus of the hash partition bound in 0002.+ int dim = hash_part? 2 : partnatts;
Call the variable as natts_per_datum or just natts?natts represents the number of attributes, but for the hash partition bound we
are not dealing with the attribute so that I have used short-form of dimension,
thoughts?
Okay, I think the dimension(dim) is also unfit here. Any suggestions?
+ hash_part? true : key->parttypbyval[j], + key->parttyplen[j]); parttyplen is the length of partition key attribute, whereas what you want here is the length of type of modulus and remainder. Is that correct? Probably we need some special handling wherever parttyplen and parttypbyval is used e.g. in call to partition_bounds_equal() from build_joinrel_partition_info().Unless I am missing something, I don't think we should worry about parttyplen
because in the datumCopy() when the datatype is pass-by-value then typelen
is ignored.Regards,
Amul
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
+ hash_part? true : key->parttypbyval[j], + key->parttyplen[j]); parttyplen is the length of partition key attribute, whereas what you want here is the length of type of modulus and remainder. Is that correct? Probably we need some special handling wherever parttyplen and parttypbyval is used e.g. in call to partition_bounds_equal() from build_joinrel_partition_info().Unless I am missing something, I don't think we should worry about parttyplen
because in the datumCopy() when the datatype is pass-by-value then typelen
is ignored.
That's true, but it's ugly, passing typbyvalue of one type and len of other.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 10, 2017 at 3:40 PM, amul sul <sulamul@gmail.com> wrote:
natts represents the number of attributes, but for the hash partition bound we
are not dealing with the attribute so that I have used short-form of dimension,
thoughts?Okay, I think the dimension(dim) is also unfit here. Any suggestions?
I think natts is ok, since we are dealing with the number of
attributes in the pack of datums; esp. when ndatums is already taken.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 10, 2017 at 3:42 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
+ hash_part? true : key->parttypbyval[j], + key->parttyplen[j]); parttyplen is the length of partition key attribute, whereas what you want here is the length of type of modulus and remainder. Is that correct? Probably we need some special handling wherever parttyplen and parttypbyval is used e.g. in call to partition_bounds_equal() from build_joinrel_partition_info().Unless I am missing something, I don't think we should worry about parttyplen
because in the datumCopy() when the datatype is pass-by-value then typelen
is ignored.That's true, but it's ugly, passing typbyvalue of one type and len of other.
How about the attached patch(0003)?
Also, the dim variable is renamed to natts.
Regards,
Amul
Attachments:
0001-partition_bounds_copy-code-refactoring-v1.patchapplication/octet-stream; name=0001-partition_bounds_copy-code-refactoring-v1.patchDownload
From 2c74400e45bf42f22eb9b8b4702d9501b8691367 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:36:24 +0530
Subject: [PATCH 1/3] partition_bounds_copy code refactoring v1
---
src/backend/catalog/partition.c | 35 +++++++++++++++++++++++++++++++++--
1 file changed, 33 insertions(+), 2 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index ebda85e4ef..b82ceb903d 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -149,6 +149,7 @@ 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);
/*
* RelationBuildPartitionDesc
@@ -721,8 +722,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);
@@ -2894,3 +2894,34 @@ 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_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;
+}
--
2.14.1
0002-hash-partitioning_another_design-v24.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v24.patchDownload
From ad439143f66ac237606f354779bb1c660c70da01 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:50:49 +0530
Subject: [PATCH 2/3] hash-partitioning_another_design-v24
v24:
- Added get_greatest_modulus() to as per Ashutosh's suggestion[1].
1] https://postgr.es/m/CAFjFpRexH08-rb2LzodCsEeVTbxRQiNuYgh7XfGJ1Gd4zg+AaQ@mail.gmail.com
v23 - v11 history:
Refer 0001-v23 patch @
https://postgr.es/m/CAAJ_b94A6NCGVkdQxuhLR8x0=20vaD9EH8mQ0Tj8X3M06Tp-Ow@mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 614 +++++++++++++++++++++++++++--
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 63 ++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 +++
src/test/regress/sql/create_table.sql | 46 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1212 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2150..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385ece7..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1420,6 +1420,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 1477288851..43baff40e5 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -363,6 +365,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,7 +511,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -497,9 +522,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provides only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1647,6 +1686,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1750,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b82ceb903d..1b894bd1be 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -15,6 +15,7 @@
#include "postgres.h"
+#include "access/hash.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -61,26 +62,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 +105,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 +129,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 +145,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 +154,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);
@@ -150,6 +172,11 @@ static int partition_bound_bsearch(PartitionKey key,
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
@@ -175,6 +202,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;
@@ -256,7 +286,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;
@@ -485,6 +543,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));
@@ -618,8 +712,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];
@@ -656,49 +749,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 or
+ * same moduli with different number of partitions, 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. This means that the index of a given partition is same as
+ * the 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -801,6 +942,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 +1395,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 +1770,127 @@ 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.
+ *
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b, HASH_SEED))
+ *
+ * where hash_fn_1_extended and hash_fn_2_extended are datatype-specific hash
+ * functions(extended version) for columns a and b respectively and HASH_SEED
+ * is a predefined 64-bit seed.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2412,6 +2762,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 +2885,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 +3103,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],
@@ -2909,6 +3311,14 @@ get_partition_bound_num_indexes(PartitionBoundInfo 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;
@@ -2925,3 +3335,121 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
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]);
+}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = ROTATE_HIGH_AND_LOW_32BITS(rowHash);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt64(hash_array[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 2d4dcd7556..a60f4a29ff 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca909..cadd253ef1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 7a700018e7..2866fd7b4a 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 2532edc94a..90995a118f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3573,6 +3573,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 07ba69178c..1331939479 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2390,6 +2390,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4c83a63f7d..92f5b86418 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..8c9e8ad158 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("default hash partition is not supported")));
+
/*
* 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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 84759b6149..ee4369652d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8708,6 +8712,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..c58d1e17b7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac0239d..de7919af45 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.h"
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
+
/*
* PartitionBoundInfo encapsulates a set of partition bounds. It is usually
* associated with partitioned tables as part of its partition descriptor.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..470b1f6482 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 50eec730b3..bde2a020e8 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index dbe438dcd4..13b049009b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3494,6 +3495,59 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: partition constraint for table "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3505,12 +3559,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3607,6 +3666,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..32ef71fc13 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,33 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: default hash partition is not supported
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +557,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +657,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -771,6 +822,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..2294427305 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..52842e30b4 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (4, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 0c8ae2ab97..972f5aae35 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2304,6 +2305,62 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2315,12 +2372,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2393,6 +2454,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..bb78df3247 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,25 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 2);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +514,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +604,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -654,6 +688,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..3a55f50c07 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..9709424186 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + 1 + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 2);
+insert into hpart4 values (12, 12);
+
+-- fail
+update hpart1 set a = 4, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da2ee..18d56417bd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
0003-Enable-partition-wise-join-support-v3.patchapplication/octet-stream; name=0003-Enable-partition-wise-join-support-v3.patchDownload
From 189a27a1ff4354764e5909df884e4e2729a68b32 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 15:08:34 +0530
Subject: [PATCH 3/3] Enable partition wise join support v3
v3:
-In partition_bounds_copy, used natts variable instead
dim in the previous version.partition_bounds_copy and
added byval & typlen variable.
v2:
-Moved switch case from partition_bounds_copy() to 0001 patch.
v1:
-Few changes to enable partition wise join support for
the hash partitioned table.
-Added regression tests.
---
src/backend/catalog/partition.c | 30 +++++++++--
src/backend/optimizer/path/joinrels.c | 10 +++-
src/test/regress/expected/partition_join.out | 81 ++++++++++++++++++++++++++++
src/test/regress/sql/partition_join.sql | 32 +++++++++++
4 files changed, 147 insertions(+), 6 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1b894bd1be..6017ca1c16 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -889,15 +889,37 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
- dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
- 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)
+ {
+ byval = true; /* modulus and remainder are integers */
+ typlen = 4; /* Don't worry; datumCopy will ignore this for
+ a pass-by-value datatype */
+ }
+ 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);
}
}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c52de..5df861f9cf 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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. */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 234b8b5381..1c8cdb34fb 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1256,6 +1256,87 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
One-Time Filter: false
(14 rows)
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ QUERY PLAN
+--------------------------------------------------------------------------------------
+ Sort
+ Sort Key: t1.c, t3.c
+ -> HashAggregate
+ Group Key: t1.c, t2.c, t3.c
+ -> Result
+ -> Append
+ -> Hash Join
+ Hash Cond: (t1.c = t2.c)
+ -> Seq Scan on pht1_p1 t1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+ -> Seq Scan on pht2_p1 t2
+ -> Hash
+ -> Seq Scan on pht1_e_p1 t3
+ -> Hash Join
+ Hash Cond: (t1_1.c = t2_1.c)
+ -> Seq Scan on pht1_p2 t1_1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+ -> Seq Scan on pht2_p2 t2_1
+ -> Hash
+ -> Seq Scan on pht1_e_p2 t3_1
+ -> Hash Join
+ Hash Cond: (t1_2.c = t2_2.c)
+ -> Seq Scan on pht1_p3 t1_2
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+ -> Seq Scan on pht2_p3 t2_2
+ -> Hash
+ -> Seq Scan on pht1_e_p3 t3_2
+(33 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ avg | avg | avg | c | c | c
+----------------------+----------------------+-----------------------+------+------+-------
+ 24.0000000000000000 | 24.0000000000000000 | 48.0000000000000000 | 0000 | 0000 | A0000
+ 74.0000000000000000 | 75.0000000000000000 | 148.0000000000000000 | 0001 | 0001 | A0001
+ 124.0000000000000000 | 124.5000000000000000 | 248.0000000000000000 | 0002 | 0002 | A0002
+ 174.0000000000000000 | 174.0000000000000000 | 348.0000000000000000 | 0003 | 0003 | A0003
+ 224.0000000000000000 | 225.0000000000000000 | 448.0000000000000000 | 0004 | 0004 | A0004
+ 274.0000000000000000 | 274.5000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
+ 324.0000000000000000 | 324.0000000000000000 | 648.0000000000000000 | 0006 | 0006 | A0006
+ 374.0000000000000000 | 375.0000000000000000 | 748.0000000000000000 | 0007 | 0007 | A0007
+ 424.0000000000000000 | 424.5000000000000000 | 848.0000000000000000 | 0008 | 0008 | A0008
+ 474.0000000000000000 | 474.0000000000000000 | 948.0000000000000000 | 0009 | 0009 | A0009
+ 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
+ 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
+(12 rows)
+
--
-- multiple levels of partitioning
--
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index ca525d9941..2316bbdcb8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -229,6 +229,38 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+
--
-- multiple levels of partitioning
--
--
2.14.1
On Tue, Oct 10, 2017 at 7:07 AM, amul sul <sulamul@gmail.com> wrote:
How about the attached patch(0003)?
Also, the dim variable is renamed to natts.
I'm not sure I believe this comment:
+ /*
+ * We arrange the partitions in the ascending order of their modulus
+ * and remainders. Also every modulus is factor of next larger
+ * modulus. This means that the index of a given partition is same as
+ * the 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.
+ */
I am particularly not sure that I believe that the index of a
partition must be the same as the remainder. It doesn't seem like
that would be true when there is more than one modulus or when some
partitions are missing.
+ 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;
+ }
+ }
I don't think this is quite right. It checks the new modulus against
prev_modulus whenever prev_modulus is defined, which is correct, but
it doesn't check the new modulus against the next_modulus except when
offset < 0. But actually that check needs to be performed, I think,
whenever the new modulus is less than the greatest modulus seen so
far.
+ * For a partitioned table defined as:
+ * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ * FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b,
HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b,
HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b,
HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ * hash_fn_2_extended(b,
HASH_SEED))
I think instead of this lengthy example you should try to explain the
general rule. Maybe something like: 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 hash
values computed for each column of the partition key using the
extended hash function from the appropriate opclass.
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
It would be nice to use the hash_combine() facility Andres recently
added for this rather than having a way to do it that is specific to
hash partitioning, but that function only works for 32-bit hash
values. Maybe we can persuade Andres to add a hash_combine64...
+ * a hash operator class
Missing period at end.
+ if (strategy == PARTITION_STRATEGY_HASH)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("default hash partition is not supported")));
Maybe errmsg("a hash-partitioned table may not have a default partition")?
+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
I suggest HASH_PARTITION_SEED -- this is too generic.
Have you checked how well the tests you've added cover the code you've
added? What code is not covered by the tests, and is there any way to
cover it?
Thanks,
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017/09/30 1:53, Robert Haas wrote:
On Thu, Sep 28, 2017 at 1:54 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:I looked into how satisfies_hash_partition() works and came up with an
idea that I think will make constraint exclusion work. What if we emitted
the hash partition constraint in the following form instead:hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
<mod>) = <rem>With that form, constraint exclusion seems to work as illustrated below:
\d+ p0
<...>
Partition constraint:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)-- note only p0 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0;What we actually want constraint exclusion to cover is SELECT * FROM p
WHERE a = 525600;
I agree.
As Amul says, nobody's going to enter a query in the form you have it
here. Life is too short to take time to put queries into bizarre
forms.
Here too. I was falsely thinking that satisfies_hash_partition() is
intended to be used for more than just enforcing the partition constraint
when data is directly inserted into a hash partition, or more precisely to
be used in the CHECK constraint of the table that is to be attached as a
hash partition. Now, we ask users to add such a constraint to avoid the
constraint validation scan, because the system knows how to infer from the
constraint that the partition constraint is satisfied. I observed however
that, unlike range and list partitioning, the hash partition's constraint
could only ever be implied because of structural equality (equal()'ness)
of the existing constraint expression and the partition constraint
expression. For example, a more restrictive range or list qual implies
the partition constraint, but it requires performing btree operator based
proof. The proof is impossible with the chosen structure of hash
partitioning constraint, but it seems that that's OK. That is, it's OK to
ask users to add the exact constraint (matching modulus and reminder
values in the call to satisfies_hash_partition() specified in the CHECK
constraint) to avoid the validation scan.
Thanks,
Amit
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Oct 12, 2017 at 6:31 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Oct 10, 2017 at 7:07 AM, amul sul <sulamul@gmail.com> wrote:
How about the attached patch(0003)?
Also, the dim variable is renamed to natts.I'm not sure I believe this comment:
+ /* + * We arrange the partitions in the ascending order of their modulus + * and remainders. Also every modulus is factor of next larger + * modulus. This means that the index of a given partition is same as + * the 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. + */I am particularly not sure that I believe that the index of a
partition must be the same as the remainder. It doesn't seem like
that would be true when there is more than one modulus or when some
partitions are missing.
Looks like an explanation by the comment is not good enough, will think on this.
Here are the links for the previous discussion:
1] /messages/by-id/CAFjFpRfHqSGBjNgJV2p+C4Yr5Qxvwygdsg4G_VQ6q9NTB-i3MA@mail.gmail.com
2] /messages/by-id/CAFjFpRdeESKFkVGgmOdYvmD3d56-58c5VCBK0zDRjHrkq_VcNg@mail.gmail.com
+ 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; + } + }I don't think this is quite right. It checks the new modulus against
prev_modulus whenever prev_modulus is defined, which is correct, but
it doesn't check the new modulus against the next_modulus except when
offset < 0. But actually that check needs to be performed, I think,
whenever the new modulus is less than the greatest modulus seen so
far.
It does. See the "if (valid_modulus && (offset + 1) < ndatums)" block in the
else part of the snippet that you are referring.
For e.g new modulus 25 & 150 is not accepted for the hash partitioned bound with
modulus 10,50,200. Will cover this test as well.
+ * For a partitioned table defined as: + * CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b); + * + * CREATE TABLE p_p1 PARTITION OF simple_hash + * FOR VALUES WITH (MODULUS 2, REMAINDER 1); + * CREATE TABLE p_p2 PARTITION OF simple_hash + * FOR VALUES WITH (MODULUS 4, REMAINDER 2); + * CREATE TABLE p_p3 PARTITION OF simple_hash + * FOR VALUES WITH (MODULUS 8, REMAINDER 0); + * CREATE TABLE p_p4 PARTITION OF simple_hash + * FOR VALUES WITH (MODULUS 8, REMAINDER 4); + * + * This function will return one of the following in the form of an + * expression: + * + * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED), + * hash_fn_2_extended(b, HASH_SEED)) + * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED), + * hash_fn_2_extended(b, HASH_SEED)) + * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED), + * hash_fn_2_extended(b, HASH_SEED)) + * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED), + * hash_fn_2_extended(b, HASH_SEED))I think instead of this lengthy example you should try to explain the
general rule. Maybe something like: 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 hash
values computed for each column of the partition key using the
extended hash function from the appropriate opclass.
Okay will add this.
+static uint64 +mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
How about combining high 32 bits and the low 32 bits separately as shown below?
static inline uint64
hash_combine64(uint64 a, uint64 b)
{
return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
| hash_combine((unit32) a, (unit32) b));
}
It would be nice to use the hash_combine() facility Andres recently
added for this rather than having a way to do it that is specific to
hash partitioning, but that function only works for 32-bit hash
values. Maybe we can persuade Andres to add a hash_combine64...+ * a hash operator class
Missing period at end.
Okay will fix this.
+ if (strategy == PARTITION_STRATEGY_HASH) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("default hash partition is not supported")));Maybe errmsg("a hash-partitioned table may not have a default partition")?
Okay will add this.
+/* Seed for the extended hash function */ +#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)I suggest HASH_PARTITION_SEED -- this is too generic.
Okay will add this.
Have you checked how well the tests you've added cover the code you've
added? What code is not covered by the tests, and is there any way to
cover it?
Will try to get gcov report for this patch.
Thanks for your review.
Regards,
Amul
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Oct 12, 2017 at 9:08 AM, amul sul <sulamul@gmail.com> wrote:
How about combining high 32 bits and the low 32 bits separately as shown below?
static inline uint64
hash_combine64(uint64 a, uint64 b)
{
return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
| hash_combine((unit32) a, (unit32) b));
}
I doubt that's the best approach, but I don't have something specific
to recommend.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017-10-12 10:05:26 -0400, Robert Haas wrote:
On Thu, Oct 12, 2017 at 9:08 AM, amul sul <sulamul@gmail.com> wrote:
How about combining high 32 bits and the low 32 bits separately as shown below?
static inline uint64
hash_combine64(uint64 a, uint64 b)
{
return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
| hash_combine((unit32) a, (unit32) b));
}I doubt that's the best approach, but I don't have something specific
to recommend.
Yea, that doesn't look great. There's basically no intermixing between
low and high 32 bits. going on. We probably should just expand the
concept of the 32 bit function:
static inline uint32
hash_combine32(uint32 a, uint32 b)
{
/* 0x9e3779b9 is the golden ratio reciprocal */
a ^= b + 0x9e3779b9 + (a << 6) + (a >> 2);
return a;
}
to something roughly like:
static inline uint64
hash_combine64(uint64 a, uint64 b)
{
/* 0x49A0F4DD15E5A8E3 is 64bit random data */
a ^= b + 0x49A0F4DD15E5A8E3 + (a << 54) + (a >> 7);
return a;
}
In contrast to the 32 bit version's fancy use of the golden ratio
reciprocal as a constant I went brute force, and just used 64bit of
/dev/random. From my understanding the important property is that bits
are independent from each other, nothing else.
The shift widths are fairly random, but they should bring in enough bit
perturbation when mixing in only 32bit of hash value (i.e
0x00000000xxxxxxxx).
Are we going to rely on the the combine function to stay the same
forever after?
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Oct 12, 2017 at 3:43 PM, Andres Freund <andres@anarazel.de> wrote:
Are we going to rely on the the combine function to stay the same
forever after?
If we change them, it will be a pg_upgrade compatibility break for
anyone using hash-partitioned tables with more than one partitioning
column. Dump and reload will also break unless
--load-via-partition-root is used.
In other words, it's not utterly fixed in stone --- we invented
--load-via-partition-root primarily to cope with circumstances that
could change hash values --- but we sure don't want to be changing it
with any regularity, or for a less-than-excellent reason.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017-10-12 16:06:11 -0400, Robert Haas wrote:
On Thu, Oct 12, 2017 at 3:43 PM, Andres Freund <andres@anarazel.de> wrote:
Are we going to rely on the the combine function to stay the same
forever after?If we change them, it will be a pg_upgrade compatibility break for
anyone using hash-partitioned tables with more than one partitioning
column. Dump and reload will also break unless
--load-via-partition-root is used.In other words, it's not utterly fixed in stone --- we invented
--load-via-partition-root primarily to cope with circumstances that
could change hash values --- but we sure don't want to be changing it
with any regularity, or for a less-than-excellent reason.
Yea, that's what I expected. It'd probably good for somebody to run
smhasher or such on the output of the combine function (or even better,
on both the 32 and 64 bit variants) in that case.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Oct 12, 2017 at 4:20 PM, Andres Freund <andres@anarazel.de> wrote:
In other words, it's not utterly fixed in stone --- we invented
--load-via-partition-root primarily to cope with circumstances that
could change hash values --- but we sure don't want to be changing it
with any regularity, or for a less-than-excellent reason.Yea, that's what I expected. It'd probably good for somebody to run
smhasher or such on the output of the combine function (or even better,
on both the 32 and 64 bit variants) in that case.
Not sure how that test suite works exactly, but presumably the
characteristics in practice will depend the behavior of the hash
functions used as input the combine function - so the behavior could
be good for an (int, int) key but bad for a (text, date) key, or
whatever.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017-10-12 17:27:52 -0400, Robert Haas wrote:
On Thu, Oct 12, 2017 at 4:20 PM, Andres Freund <andres@anarazel.de> wrote:
In other words, it's not utterly fixed in stone --- we invented
--load-via-partition-root primarily to cope with circumstances that
could change hash values --- but we sure don't want to be changing it
with any regularity, or for a less-than-excellent reason.Yea, that's what I expected. It'd probably good for somebody to run
smhasher or such on the output of the combine function (or even better,
on both the 32 and 64 bit variants) in that case.Not sure how that test suite works exactly, but presumably the
characteristics in practice will depend the behavior of the hash
functions used as input the combine function - so the behavior could
be good for an (int, int) key but bad for a (text, date) key, or
whatever.
I don't think that's true, unless you have really bad hash functions on
the the component hashes. A hash combine function can't really do
anything about badly hashed input, what you want is that it doesn't
*reduce* the quality of the hash by combining.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 10, 2017 at 4:37 PM, amul sul <sulamul@gmail.com> wrote:
On Tue, Oct 10, 2017 at 3:42 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
+ hash_part? true : key->parttypbyval[j], + key->parttyplen[j]); parttyplen is the length of partition key attribute, whereas what you want here is the length of type of modulus and remainder. Is that correct? Probably we need some special handling wherever parttyplen and parttypbyval is used e.g. in call to partition_bounds_equal() from build_joinrel_partition_info().Unless I am missing something, I don't think we should worry about parttyplen
because in the datumCopy() when the datatype is pass-by-value then typelen
is ignored.That's true, but it's ugly, passing typbyvalue of one type and len of other.
How about the attached patch(0003)?
Also, the dim variable is renamed to natts.
Probably we should move changes to partition_bounds_copy() in 0003 to
0001, whose name suggests so.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Oct 16, 2017 at 2:36 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Probably we should move changes to partition_bounds_copy() in 0003 to
0001, whose name suggests so.
We can't do this, hash partition strategy is introduced by 0002. Sorry
for the noise.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Oct 13, 2017 at 3:00 AM, Andres Freund <andres@anarazel.de> wrote:
On 2017-10-12 17:27:52 -0400, Robert Haas wrote:
On Thu, Oct 12, 2017 at 4:20 PM, Andres Freund <andres@anarazel.de> wrote:
In other words, it's not utterly fixed in stone --- we invented
--load-via-partition-root primarily to cope with circumstances that
could change hash values --- but we sure don't want to be changing it
with any regularity, or for a less-than-excellent reason.Yea, that's what I expected. It'd probably good for somebody to run
smhasher or such on the output of the combine function (or even better,
on both the 32 and 64 bit variants) in that case.Not sure how that test suite works exactly, but presumably the
characteristics in practice will depend the behavior of the hash
functions used as input the combine function - so the behavior could
be good for an (int, int) key but bad for a (text, date) key, or
whatever.I don't think that's true, unless you have really bad hash functions on
the the component hashes. A hash combine function can't really do
anything about badly hashed input, what you want is that it doesn't
*reduce* the quality of the hash by combining.
I tried to get suggested SMHasher[1] test result for the hash_combine
for 32-bit and 64-bit version.
SMHasher works on hash keys of the form {0}, {0,1}, {0,1,2}... up to
N=255, using 256-N as the seed, for the hash_combine testing we
needed two hash value to be combined, for that, I've generated 64
and 128-bit hash using cityhash functions[2] for the given smhasher
key then split in two part to test 32-bit and 64-bit hash_combine
function respectively. Attached patch for SMHasher code changes &
output of 32-bit and 64-bit hash_combine testing. Note that I have
skipped speed test this test which is irrelevant here.
By referring other hash function results [3], we can see that hash_combine
test results are not bad either.
Do let me know if current testing is not good enough or if you want me to do
more testing, thanks.
1] https://github.com/aappleby/smhasher
2] https://github.com/aappleby/smhasher/blob/master/src/CityTest.cpp
3] https://github.com/rurban/smhasher/tree/master/doc
Regards,
Amul
Attachments:
0001-add-hash_combine-functions-in-SMHasher.patchapplication/octet-stream; name=0001-add-hash_combine-functions-in-SMHasher.patchDownload
From 16dadbc8371c84184b36706131d5cfed9502f699 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 23 Oct 2017 18:32:48 +0530
Subject: [PATCH] add hash_combine functions in SMHasher
---
src/CityTest.cpp | 30 ++++++++++++++++++++++++++++++
src/Hashes.h | 2 ++
src/PMurHash.c | 1 -
src/main.cpp | 2 ++
4 files changed, 34 insertions(+), 1 deletion(-)
diff --git a/src/CityTest.cpp b/src/CityTest.cpp
index 4190cc8..0259724 100644
--- a/src/CityTest.cpp
+++ b/src/CityTest.cpp
@@ -13,3 +13,33 @@ void CityHash128_test ( const void * key, int len, uint32_t seed, void * out )
*(uint128*)out = CityHash128WithSeed((const char*)key,len,s);
}
+
+void hash_combine32(const void *key, int len, uint32_t seed, void *out)
+{
+ uint32_t a;
+ uint32_t b;
+ uint64 h;
+
+ CityHash64_test(key, len, seed, &h);
+ a = (uint32_t) (h);
+ b = (uint32_t) (h >> 32);
+
+ /* 0x9e3779b9 is the golden ratio reciprocal */
+ a ^= b + 0x9e3779b9 + (a << 6) + (a >> 2);
+
+ *(uint32_t*)out = a;
+}
+void hash_combine64(const void *key, int len, uint32_t seed, void *out)
+{
+ uint64 a;
+ uint64 b;
+ uint128 h;
+
+ CityHash128_test(key, len, seed, &h);
+ a = Uint128Low64(h);
+ b = Uint128High64(h);
+
+ a ^= b + 0x49A0F4DD15E5A8E3 + (a << 54) + (a >> 7);
+
+ *(uint64*)out = a;
+}
diff --git a/src/Hashes.h b/src/Hashes.h
index 6c04ae1..951be61 100644
--- a/src/Hashes.h
+++ b/src/Hashes.h
@@ -36,6 +36,8 @@ void MurmurOAAT_test ( const void * key, int len, uint32_t seed, void * ou
void Crap8_test ( const void * key, int len, uint32_t seed, void * out );
void CityHash128_test ( const void * key, int len, uint32_t seed, void * out );
void CityHash64_test ( const void * key, int len, uint32_t seed, void * out );
+void hash_combine32 ( const void * key, int len, uint32_t seed, void * out );
+void hash_combine64 ( const void * key, int len, uint32_t seed, void * out );
void SpookyHash32_test ( const void * key, int len, uint32_t seed, void * out );
void SpookyHash64_test ( const void * key, int len, uint32_t seed, void * out );
diff --git a/src/PMurHash.c b/src/PMurHash.c
index 0175012..8697a5d 100644
--- a/src/PMurHash.c
+++ b/src/PMurHash.c
@@ -313,5 +313,4 @@ void PMurHash32_test(const void *key, int len, uint32_t seed, void *out)
h1 = PMurHash32_Result(h1, carry, len);
*(uint32_t*)out = h1;
}
-
/*---------------------------------------------------------------------------*/
diff --git a/src/main.cpp b/src/main.cpp
index 678ddb2..7823acb 100644
--- a/src/main.cpp
+++ b/src/main.cpp
@@ -80,6 +80,8 @@ HashInfo g_hashes[] =
{ MurmurHash3_x64_128, 128, 0x6384BA69, "Murmur3F", "MurmurHash3 for x64, 128-bit" },
{ PMurHash32_test, 32, 0xB0F57EE3, "PMurHash32", "Shane Day's portable-ized MurmurHash3 for x86, 32-bit." },
+ { hash_combine32, 32, 0x83D5F11A, "hash_combine32", "test hash combine 32 pg function." },
+ { hash_combine64, 64, 0x3B439A64, "hash_combine64", "test hash combine 64 pg function." },
};
HashInfo * findHash ( const char * name )
--
2.14.1
On Thu, Oct 12, 2017 at 6:38 PM, amul sul <sulamul@gmail.com> wrote:
On Thu, Oct 12, 2017 at 6:31 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Oct 10, 2017 at 7:07 AM, amul sul <sulamul@gmail.com> wrote:
How about the attached patch(0003)?
Also, the dim variable is renamed to natts.I'm not sure I believe this comment:
+ /* + * We arrange the partitions in the ascending order of their modulus + * and remainders. Also every modulus is factor of next larger + * modulus. This means that the index of a given partition is same as + * the 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. + */I am particularly not sure that I believe that the index of a
partition must be the same as the remainder. It doesn't seem like
that would be true when there is more than one modulus or when some
partitions are missing.Looks like an explanation by the comment is not good enough, will think on this.
Here are the links for the previous discussion:
1] /messages/by-id/CAFjFpRfHqSGBjNgJV2p+C4Yr5Qxvwygdsg4G_VQ6q9NTB-i3MA@mail.gmail.com
2] /messages/by-id/CAFjFpRdeESKFkVGgmOdYvmD3d56-58c5VCBK0zDRjHrkq_VcNg@mail.gmail.com
I have modified the comment little bit, now let me explain the theory behind it.
rd_partdesc->boundinfo->indexes array stores an index in rd_partdesc->oids
array corresponding to a given partition falls at the positions. And position in
indexes array is decided using remainder + N * modulus_of_that_partition
(where N = 0,1,2,..,).
For the case where the same modulus, the remainder will be 0,1,2,..,
and the index of that partition will be at 0,1,2,..,. (N=0).
For the case where more than one modulus then an index of a partition oid in the
oids array could be stored at the multiple places in indexes array if
its modulus is < greatest_modulus amongst bound (where N = 0,1,2,..,).
For example, partition bound (Modulus, remainder) = p1(2,0), p2(4,1),
p3(8,3), p4(8,7) Oids array [p1,p2,p3,p4] sorted by Modulus and then
by remainder and indexes array [0, 1, 0, 3, 0, 1, 0, 4] size of indexes
array is greatest_modulus.
In other word, if a partition index in oids array in the indexes array is
stored multiple times, then the lowest of the differences between them
is the modulus of that partition. In above case for the partition p1, index
in oids array stored at 0,2,4,6. You can see lowest is the remainder and
minimum difference is the modulus of p1.
Since indexes arrays in both the bounds are same, for a given index in oids
array, the positions where it falls is same for both bounds. One can argue that
two different moduli could have the same remainder position, which is
not allowed
because that will cause partition overlap error at creation and also we have a
restriction on modulus that each modulus in the hash partition bound should be
the factor of next modulus.
[....]
+static uint64 +mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)How about combining high 32 bits and the low 32 bits separately as shown below?
static inline uint64
hash_combine64(uint64 a, uint64 b)
{
return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
| hash_combine((unit32) a, (unit32) b));
}
I have used hash_combine64 function suggested by Andres [1].
[....]
Have you checked how well the tests you've added cover the code you've
added? What code is not covered by the tests, and is there any way to
cover it?Will try to get gcov report for this patch.
Tests in the attached patch covers almost all the code expect few[2].
Updated patch attached.
1] /messages/by-id/20171012194353.3nealiykmjura4bi@alap3.anarazel.de
2] Refer gcov_output.txt attachment.
Regards,
Amul Sul
Attachments:
0001-partition_bounds_copy-code-refactoring-v1.patchapplication/octet-stream; name=0001-partition_bounds_copy-code-refactoring-v1.patchDownload
From 2c74400e45bf42f22eb9b8b4702d9501b8691367 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:36:24 +0530
Subject: [PATCH 1/3] partition_bounds_copy code refactoring v1
---
src/backend/catalog/partition.c | 35 +++++++++++++++++++++++++++++++++--
1 file changed, 33 insertions(+), 2 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index ebda85e4ef..b82ceb903d 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -149,6 +149,7 @@ 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);
/*
* RelationBuildPartitionDesc
@@ -721,8 +722,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);
@@ -2894,3 +2894,34 @@ 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_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;
+}
--
2.14.1
0002-hash-partitioning_another_design-v25.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v25.patchDownload
From a2ffcc7a1cbc32df223994115be71f159b8f9f05 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:50:49 +0530
Subject: [PATCH 2/3] hash-partitioning_another_design-v25
v25:
Changes w.r.t Robert's review comments[1]
1] https://postgr.es/m/CA+TgmoYhB0+r--=bYkvexS7h+6P9GdoO8BN+vFWUuFxTUArZXQ@mail.gmail.com
v24:
- Added get_greatest_modulus() to as per Ashutosh's suggestion[1].
1] https://postgr.es/m/CAFjFpRexH08-rb2LzodCsEeVTbxRQiNuYgh7XfGJ1Gd4zg+AaQ@mail.gmail.com
v23 - v11 history:
Refer 0001-v23 patch @
https://postgr.es/m/CAAJ_b94A6NCGVkdQxuhLR8x0=20vaD9EH8mQ0Tj8X3M06Tp-Ow@mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 80 +++-
src/backend/catalog/partition.c | 588 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 15 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/include/utils/hashutils.h | 7 +
src/test/regress/expected/alter_table.out | 62 +++
src/test/regress/expected/create_table.out | 83 +++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
27 files changed, 1218 insertions(+), 96 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b05a9c2150..e38d8fc0a0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 0fb385ece7..b5fb93edac 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1420,6 +1420,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 1477288851..43baff40e5 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="PARAMETER">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="PARAMETER">numeric_literal</replaceable>, REMAINDER <replaceable class="PARAMETER">numeric_literal</replaceable> )
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -363,6 +365,29 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,7 +511,7 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
@@ -497,9 +522,23 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
include multiple columns or expressions (up to 32, but this limit can be
altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provides only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
</para>
<para>
@@ -1647,6 +1686,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1750,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH(MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b82ceb903d..f16543463f 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -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);
@@ -150,6 +173,11 @@ static int partition_bound_bsearch(PartitionKey key,
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
@@ -175,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;
@@ -256,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;
@@ -485,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));
@@ -618,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];
@@ -656,49 +750,98 @@ 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 or
+ * same moduli with different number of partitions, 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -801,6 +944,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 +1397,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 +1772,105 @@ 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
+ * hash values computed for each column of the partition key using the extended
+ * hash function from the appropriate opclass.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_PARTITION_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2412,6 +2742,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 +2865,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 +3083,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],
@@ -2909,6 +3291,14 @@ get_partition_bound_num_indexes(PartitionBoundInfo 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;
@@ -2925,3 +3315,115 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
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]);
+}
+
+/*
+ * mix_hash_value
+ *
+ * This function takes an already computed hash values and combine them
+ * into a single 64-bit value.
+ */
+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
+{
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ rowHash = hash_combine64(rowHash, DatumGetUInt64(hash_array[i]));
+ }
+
+ return rowHash;
+}
+
+/*
+ * 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;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ Datum seed = UInt64GetDatum(HASH_PARTITION_SEED);
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ 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_array[i] = FunctionCall2(&key->partsupfunc[i], values[i],
+ seed);
+ }
+ }
+
+ /* Form a single 64-bit hash value */
+ return mix_hash_value(nkeys, hash_array, isnull);
+}
+
+/*
+ * 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 mix_hash_value.
+ *
+ * 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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint64 rowHash;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ isnull[i] = PG_ARGISNULL(i + 2);
+
+ if (!isnull[i])
+ hash_array[i] = PG_GETARG_DATUM(i + 2);
+ }
+
+ /* Form a single 64-bit hash value */
+ rowHash = mix_hash_value(nkeys, hash_array, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 2d4dcd7556..f5d8ba929b 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13269,7 +13269,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;
@@ -13339,10 +13341,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)
@@ -13502,25 +13506,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca909..cadd253ef1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 7a700018e7..2866fd7b4a 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 2532edc94a..90995a118f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3573,6 +3573,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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 07ba69178c..1331939479 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2390,6 +2390,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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4c83a63f7d..92f5b86418 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..2af50419f3 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 84759b6149..ee4369652d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8708,6 +8712,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..c58d1e17b7 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,17 +943,14 @@ 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);
+ procnum);
if (!OidIsValid(funcid)) /* should not happen */
elog(ERROR, "missing support function %d(%u,%u) in opfamily %u",
- BTORDER_PROC, opclassform->opcintype, opclassform->opcintype,
+ procnum, opclassform->opcintype, opclassform->opcintype,
opclassform->opcfamily);
fmgr_info(funcid, &key->partsupfunc[i]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac0239d..8acc01a876 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.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.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..470b1f6482 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 1007" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 50eec730b3..bde2a020e8 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/include/utils/hashutils.h b/src/include/utils/hashutils.h
index 366bd0e78b..d56fc7dd61 100644
--- a/src/include/utils/hashutils.h
+++ b/src/include/utils/hashutils.h
@@ -20,6 +20,13 @@ hash_combine(uint32 a, uint32 b)
return a;
}
+static inline uint64
+hash_combine64(uint64 a, uint64 b)
+{
+ /* 0x49a0f4dd15e5a8e3 is 64bit random data */
+ a ^= b + 0x49a0f4dd15e5a8e3 + (a << 54) + (a >> 7);
+ return a;
+}
/*
* Simple inline murmur hash implementation hashing a 32 bit integer, for
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index dbe438dcd4..13b049009b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3297,6 +3297,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3494,6 +3495,59 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: partition constraint for table "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3505,12 +3559,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3607,6 +3666,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..5019d1caaf 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,37 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 0);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 1);
+CREATE TABLE hpart_3 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 200, REMAINDER 2);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 3);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- previous modulus 50 is factor of 150 but this modulus is not factor of next modulus 200.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 150, REMAINDER 3);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: a hash-partitioned table may not have a default partition
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +561,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +661,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -694,6 +749,14 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Partitions: part_c_1_10 FOR VALUES FROM (1) TO (10)
+\d+ hpart_1
+ Table "public.hpart_1"
+ Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
+--------+---------+-----------+----------+---------+---------+--------------+-------------
+ a | integer | | | | plain | |
+Partition of: hash_parted FOR VALUES WITH (modulus 10, remainder 0)
+Partition constraint: satisfies_hash_partition(10, 0, hashint4extended(a, '8816678312871386365'::bigint))
+
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
Table "public.part_c_1_10"
@@ -721,6 +784,14 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Number of partitions: 3 (Use \d+ to list them.)
+\d hash_parted
+ Table "public.hash_parted"
+ Column | Type | Collation | Nullable | Default
+--------+---------+-----------+----------+---------
+ a | integer | | |
+Partition key: HASH (a)
+Number of partitions: 3 (Use \d+ to list them.)
+
-- check that we get the expected partition constraints
CREATE TABLE range_parted4 (a int, b int, c int) PARTITION BY RANGE (abs(a), abs(b), c);
CREATE TABLE unbounded_range_part PARTITION OF range_parted4 FOR VALUES FROM (MINVALUE, MINVALUE, MINVALUE) TO (MAXVALUE, MAXVALUE, MAXVALUE);
@@ -771,6 +842,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..9d84ba4658 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..a4fe96112e 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 5);
+insert into hpart4 values (3, 4);
+-- fail
+update hpart1 set a = 3, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (3, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 0c8ae2ab97..972f5aae35 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2111,6 +2111,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2304,6 +2305,62 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2315,12 +2372,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2393,6 +2454,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..b08b33d515 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,28 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 0);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 1);
+CREATE TABLE hpart_3 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 200, REMAINDER 2);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 3);
+-- previous modulus 50 is factor of 150 but this modulus is not factor of next modulus 200.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 150, REMAINDER 3);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +517,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +607,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -629,6 +666,7 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- Both partition bound and partition key in describe output
\d+ part_c
+\d+ hpart_1
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
@@ -638,6 +676,7 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- output could vary depending on the order in which partition oids are
-- returned.
\d parted
+\d hash_parted
-- check that we get the expected partition constraints
CREATE TABLE range_parted4 (a int, b int, c int) PARTITION BY RANGE (abs(a), abs(b), c);
@@ -654,6 +693,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..791817ba50 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..0c70d64a89 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 5);
+insert into hpart4 values (3, 4);
+
+-- fail
+update hpart1 set a = 3, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 8ce97da2ee..18d56417bd 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1563,6 +1563,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
0003-Enable-partition-wise-join-support-v3.patchapplication/octet-stream; name=0003-Enable-partition-wise-join-support-v3.patchDownload
From fa6b68d79ec482446abe890cca8302a7df3a85b5 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 15:08:34 +0530
Subject: [PATCH 3/3] Enable partition wise join support v3
v3:
-In partition_bounds_copy, used natts variable instead
dim in the previous version.partition_bounds_copy and
added byval & typlen variable.
v2:
-Moved switch case from partition_bounds_copy() to 0001 patch.
v1:
-Few changes to enable partition wise join support for
the hash partitioned table.
-Added regression tests.
---
src/backend/catalog/partition.c | 30 +++++++++--
src/backend/optimizer/path/joinrels.c | 10 +++-
src/test/regress/expected/partition_join.out | 81 ++++++++++++++++++++++++++++
src/test/regress/sql/partition_join.sql | 32 +++++++++++
4 files changed, 147 insertions(+), 6 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index f16543463f..79e27d46ba 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -891,15 +891,37 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
- dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
- 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)
+ {
+ byval = true; /* modulus and remainder are integers */
+ typlen = 4; /* Don't worry; datumCopy will ignore this for
+ a pass-by-value datatype */
+ }
+ 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);
}
}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c52de..5df861f9cf 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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. */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 234b8b5381..1c8cdb34fb 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1256,6 +1256,87 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
One-Time Filter: false
(14 rows)
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ QUERY PLAN
+--------------------------------------------------------------------------------------
+ Sort
+ Sort Key: t1.c, t3.c
+ -> HashAggregate
+ Group Key: t1.c, t2.c, t3.c
+ -> Result
+ -> Append
+ -> Hash Join
+ Hash Cond: (t1.c = t2.c)
+ -> Seq Scan on pht1_p1 t1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+ -> Seq Scan on pht2_p1 t2
+ -> Hash
+ -> Seq Scan on pht1_e_p1 t3
+ -> Hash Join
+ Hash Cond: (t1_1.c = t2_1.c)
+ -> Seq Scan on pht1_p2 t1_1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+ -> Seq Scan on pht2_p2 t2_1
+ -> Hash
+ -> Seq Scan on pht1_e_p2 t3_1
+ -> Hash Join
+ Hash Cond: (t1_2.c = t2_2.c)
+ -> Seq Scan on pht1_p3 t1_2
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+ -> Seq Scan on pht2_p3 t2_2
+ -> Hash
+ -> Seq Scan on pht1_e_p3 t3_2
+(33 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ avg | avg | avg | c | c | c
+----------------------+----------------------+-----------------------+------+------+-------
+ 24.0000000000000000 | 24.0000000000000000 | 48.0000000000000000 | 0000 | 0000 | A0000
+ 74.0000000000000000 | 75.0000000000000000 | 148.0000000000000000 | 0001 | 0001 | A0001
+ 124.0000000000000000 | 124.5000000000000000 | 248.0000000000000000 | 0002 | 0002 | A0002
+ 174.0000000000000000 | 174.0000000000000000 | 348.0000000000000000 | 0003 | 0003 | A0003
+ 224.0000000000000000 | 225.0000000000000000 | 448.0000000000000000 | 0004 | 0004 | A0004
+ 274.0000000000000000 | 274.5000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
+ 324.0000000000000000 | 324.0000000000000000 | 648.0000000000000000 | 0006 | 0006 | A0006
+ 374.0000000000000000 | 375.0000000000000000 | 748.0000000000000000 | 0007 | 0007 | A0007
+ 424.0000000000000000 | 424.5000000000000000 | 848.0000000000000000 | 0008 | 0008 | A0008
+ 474.0000000000000000 | 474.0000000000000000 | 948.0000000000000000 | 0009 | 0009 | A0009
+ 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
+ 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
+(12 rows)
+
--
-- multiple levels of partitioning
--
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index ca525d9941..2316bbdcb8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -229,6 +229,38 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+
--
-- multiple levels of partitioning
--
--
2.14.1
On 2017-10-24 12:43:12 +0530, amul sul wrote:
I tried to get suggested SMHasher[1] test result for the hash_combine
for 32-bit and 64-bit version.SMHasher works on hash keys of the form {0}, {0,1}, {0,1,2}... up to
N=255, using 256-N as the seed, for the hash_combine testing we
needed two hash value to be combined, for that, I've generated 64
and 128-bit hash using cityhash functions[2] for the given smhasher
key then split in two part to test 32-bit and 64-bit hash_combine
function respectively. Attached patch for SMHasher code changes &
output of 32-bit and 64-bit hash_combine testing. Note that I have
skipped speed test this test which is irrelevant here.By referring other hash function results [3], we can see that hash_combine
test results are not bad either.Do let me know if current testing is not good enough or if you want me to do
more testing, thanks.
This looks very good! Both the tests you did, and the results for
hash_combineXX. I therefore think we can go ahead with that formulation
of hash_combine64?
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 24, 2017 at 5:00 PM, Andres Freund <andres@anarazel.de> wrote:
On 2017-10-24 12:43:12 +0530, amul sul wrote:
I tried to get suggested SMHasher[1] test result for the hash_combine
for 32-bit and 64-bit version.SMHasher works on hash keys of the form {0}, {0,1}, {0,1,2}... up to
N=255, using 256-N as the seed, for the hash_combine testing we
needed two hash value to be combined, for that, I've generated 64
and 128-bit hash using cityhash functions[2] for the given smhasher
key then split in two part to test 32-bit and 64-bit hash_combine
function respectively. Attached patch for SMHasher code changes &
output of 32-bit and 64-bit hash_combine testing. Note that I have
skipped speed test this test which is irrelevant here.By referring other hash function results [3], we can see that hash_combine
test results are not bad either.Do let me know if current testing is not good enough or if you want me to do
more testing, thanks.This looks very good! Both the tests you did, and the results for
hash_combineXX. I therefore think we can go ahead with that formulation
of hash_combine64?
Thanks, Andres. Yes we can, I've added your suggested hash_combine64 in
the latest patch[1].
Regards,
Amul
1] /messages/by-id/CAAJ_b97R2rJinGPAVmZZzpNV=-5BgYFxDfY9HYdM1bCYJFGmQw@mail.gmail.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 24, 2017 at 1:21 PM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached.
This patch needs a rebase.
It appears that satisfies_hash_func is declared incorrectly in
pg_proc.h. ProcedureCreate seems to think that provariadic should be
ANYOID if the type of the last element is ANYOID, ANYELEMENTOID if the
type of the last element is ANYARRAYOID, and otherwise the element
type corresponding to the array type. But here you have the last
element as int4[] but provariadic is any. I wrote the following query
to detect problems of this type, and I think we might want to just go
ahead and add this to the regression test suite, verifying that it
returns no rows:
select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
from pg_proc where provariadic != 0
and case proargtypes[array_length(proargtypes, 1)-1]
when 2276 then 2276 -- any -> any
when 2277 then 2283 -- anyarray -> anyelement
else (select t.oid from pg_type t where t.typarray =
proargtypes[array_length(proargtypes, 1)-1]) end
!= provariadic;
The simple fix is change provariadic to int4 and call it good. It's
tempting to go the other way and actually make it
satisfies_hash_partition(int4, int4, variadic "any"), passing the
column values directly and letting satisfies_hash_partition doing the
hashing itself. Any arguments that had a partition key type different
from the column type would have a RelabelType node placed on top of
the column, so that get_fn_expr_argtype would return the partition key
type. Then, the function could look up the hash function for that
type and call it directly on the value. That way, we'd be doing only
one function call instead of many, and the partition constraint would
look nicer in \d+ output, too. :-) On the other hand, that would
also mean that we'd have to look up the extended hash function every
time through this function, though maybe that could be prevented by
using fn_extra to cache FmgrInfos for all the hash functions on the
first time through. I'm not sure how that would compare in terms of
speed with what you have now, but maybe it's worth trying.
The second paragraph of the CREATE TABLE documentation for PARTITION
OF needs to be updated like this: "The form with <literal>IN</literal>
is used for list partitioning, the form with <literal>FROM</literal>
and <literal>TO</literal> is used for range partitioning, and the form
with <literal>WITH</literal> is used for hash partitioning."
The CREATE TABLE documentation says "When using range partitioning,
the partition key can include multiple columns or expressions (up to
32,"; this should be changed to say "When using range or hash
partitioning".
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the
datatype will
- be used. If there is none, an error will be reported.
+ expression. If no operator class is specified when creating a
partitioned
+ table, the default operator class of the appropriate type (btree for list
+ and range partitioning, hash for hash partitioning) will be used. If
+ there is none, an error will be reported.
+ </para>
+
+ <para>
+ Since hash operator class provides only equality, not ordering, collation
+ is not relevant for hash partitioning. The behaviour will be unaffected
+ if a collation is specified.
+ </para>
+
+ <para>
+ Hash partitioning will use support function 2 routines from the operator
+ class. If there is none, an error will be reported. See <xref
+ linkend="xindex-support"> for details of operator class support
+ functions.
I think we should rework this a little more heavily. I suggest the
following, starting after "a single column or expression":
<para>
Range and list partitioning require a btree operator class, while hash
partitioning requires a hash operator class. If no operator class is
specified explicitly, the default operator class of the appropriate
type will be used; if no default operator class exists, an error will
be raised. When hash partitioning is used, the operator class used
must implement support function 2 (see <xref linkend="xindex-support">
for details).
</para>
I think we can leave out the part about collations. It's possibly
worth a longer explanation here at some point: for range partitioning,
collation can affect which rows go into which partitions; for list
partitioning, it can't, but it can affect the order in which
partitions are expanded (which is a can of worms I'm not quite ready
to try to explain in user-facing documentation); for hash
partitioning, it makes no difference at all. Although at some point
we may want to document this, I think it's a job for a separate patch,
since (1) the existing documentation doesn't document the precise
import of collations on existing partitioning types and (2) I'm not
sure that CREATE TABLE is really the best place to explain this.
The example commands for creating a hash-partitioned table are missing
spaces between WITH and the parenthesis which follows.
In 0003, the changes to partition_bounds_copy claim that I shouldn't
worry about the fact that typlen is set to 4 because datumCopy won't
use it for a pass-by-value datatype, but I think that calling
functions with incorrect arguments and hoping that they ignore them
and therefore nothing bad happens doesn't sound like a very good idea.
Fortunately, I think the actual code is fine; I think we just need to
change the comments. For hash partitioning, the datums array always
contains two integers, which are of type int4, which is indeed a
pass-by-value type of length 4 (note that if we were using int8 for
the modulus and remainder, we'd need to set byval to FLOAT8PASSBYVAL).
I would just write this as:
if (hash_part)
{
typlen = sizeof(int32); /* always int4 */
byval = true; /* int4 is pass-by-value */
}
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ rowHash = hash_combine64(rowHash,
DatumGetUInt64(hash_array[i]));
+ }
Excess braces.
I think it might be better to inline the logic in mix_hash_value()
into each of the two callers. Then, the callers wouldn't need Datum
hash_array[PARTITION_MAX_KEYS]; they could just fold each new hash
value into a uint64 value. That seems likely to be slightly faster
and I don't see any real downside.
rhaas=# create table natch (a citext, b text) partition by hash (a);
ERROR: XX000: missing support function 2(16398,16398) in opfamily 16437
LOCATION: RelationBuildPartitionKey, relcache.c:954
It shouldn't be possible to reach an elog() from SQL, and this is not
a friendly error message.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sun, Oct 29, 2017 at 12:38 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Oct 24, 2017 at 1:21 PM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached.
This patch needs a rebase.
Sure, thanks a lot for your review.
It appears that satisfies_hash_func is declared incorrectly in
pg_proc.h. ProcedureCreate seems to think that provariadic should be
ANYOID if the type of the last element is ANYOID, ANYELEMENTOID if the
type of the last element is ANYARRAYOID, and otherwise the element
type corresponding to the array type. But here you have the last
element as int4[] but provariadic is any.
Actually, int4[] is also inappropriate type as we have started using a 64bit
hash function. We need something int8[] which is not available, so that I
have used ANYARRAYOID in the attached patch(0004).
I wrote the following query
to detect problems of this type, and I think we might want to just go
ahead and add this to the regression test suite, verifying that it
returns no rows:select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
from pg_proc where provariadic != 0
and case proargtypes[array_length(proargtypes, 1)-1]
when 2276 then 2276 -- any -> any
when 2277 then 2283 -- anyarray -> anyelement
else (select t.oid from pg_type t where t.typarray =
proargtypes[array_length(proargtypes, 1)-1]) end
!= provariadic;
Added in 0001 patch.
The simple fix is change provariadic to int4 and call it good. It's
tempting to go the other way and actually make it
satisfies_hash_partition(int4, int4, variadic "any"), passing the
column values directly and letting satisfies_hash_partition doing the
hashing itself. Any arguments that had a partition key type different
from the column type would have a RelabelType node placed on top of
the column, so that get_fn_expr_argtype would return the partition key
type. Then, the function could look up the hash function for that
type and call it directly on the value. That way, we'd be doing only
one function call instead of many, and the partition constraint would
look nicer in \d+ output, too. :-) On the other hand, that would
also mean that we'd have to look up the extended hash function every
time through this function, though maybe that could be prevented by
using fn_extra to cache FmgrInfos for all the hash functions on the
first time through. I'm not sure how that would compare in terms of
speed with what you have now, but maybe it's worth trying.
One advantage of current implementation is that we can see which hash
function are used for the each partitioning column and also we don't need to
worry about user specified opclass and different input types.
Something similar I've tried in my initial patch version[1], but I have missed
user specified opclass handling for each partitioning column. Do you want me
to handle opclass using RelabelType node? I am afraid that, that would make
the \d+ output more horrible than the current one if non-default opclass used.
The second paragraph of the CREATE TABLE documentation for PARTITION
OF needs to be updated like this: "The form with <literal>IN</literal>
is used for list partitioning, the form with <literal>FROM</literal>
and <literal>TO</literal> is used for range partitioning, and the form
with <literal>WITH</literal> is used for hash partitioning."
Fixed in the attached version(0004).
The CREATE TABLE documentation says "When using range partitioning,
the partition key can include multiple columns or expressions (up to
32,"; this should be changed to say "When using range or hash
partitioning".
Fixed in the attached version(0004).
- expression. If no B-tree operator class is specified when creating a - partitioned table, the default B-tree operator class for the datatype will - be used. If there is none, an error will be reported. + expression. If no operator class is specified when creating a partitioned + table, the default operator class of the appropriate type (btree for list + and range partitioning, hash for hash partitioning) will be used. If + there is none, an error will be reported. + </para> + + <para> + Since hash operator class provides only equality, not ordering, collation + is not relevant for hash partitioning. The behaviour will be unaffected + if a collation is specified. + </para> + + <para> + Hash partitioning will use support function 2 routines from the operator + class. If there is none, an error will be reported. See <xref + linkend="xindex-support"> for details of operator class support + functions.I think we should rework this a little more heavily. I suggest the
following, starting after "a single column or expression":<para>
Range and list partitioning require a btree operator class, while hash
partitioning requires a hash operator class. If no operator class is
specified explicitly, the default operator class of the appropriate
type will be used; if no default operator class exists, an error will
be raised. When hash partitioning is used, the operator class used
must implement support function 2 (see <xref linkend="xindex-support">
for details).
</para>
Thanks again, added in the attached version(0004).
I think we can leave out the part about collations. It's possibly
worth a longer explanation here at some point: for range partitioning,
collation can affect which rows go into which partitions; for list
partitioning, it can't, but it can affect the order in which
partitions are expanded (which is a can of worms I'm not quite ready
to try to explain in user-facing documentation); for hash
partitioning, it makes no difference at all. Although at some point
we may want to document this, I think it's a job for a separate patch,
since (1) the existing documentation doesn't document the precise
import of collations on existing partitioning types and (2) I'm not
sure that CREATE TABLE is really the best place to explain this.
Okay.
The example commands for creating a hash-partitioned table are missing
spaces between WITH and the parenthesis which follows.
Fixed in the attached version(0004).
In 0003, the changes to partition_bounds_copy claim that I shouldn't
worry about the fact that typlen is set to 4 because datumCopy won't
use it for a pass-by-value datatype, but I think that calling
functions with incorrect arguments and hoping that they ignore them
and therefore nothing bad happens doesn't sound like a very good idea.
Fortunately, I think the actual code is fine; I think we just need to
change the comments. For hash partitioning, the datums array always
contains two integers, which are of type int4, which is indeed a
pass-by-value type of length 4 (note that if we were using int8 for
the modulus and remainder, we'd need to set byval to FLOAT8PASSBYVAL).
I would just write this as:if (hash_part)
{
typlen = sizeof(int32); /* always int4 */
byval = true; /* int4 is pass-by-value */
}
Fixed in the attached version (now patch number is 0005).
+ for (i = 0; i < nkeys; i++) + { + if (!isnull[i]) + rowHash = hash_combine64(rowHash, DatumGetUInt64(hash_array[i])); + }Excess braces.
Fixed in the attached version(0004).
I think it might be better to inline the logic in mix_hash_value()
into each of the two callers. Then, the callers wouldn't need Datum
hash_array[PARTITION_MAX_KEYS]; they could just fold each new hash
value into a uint64 value. That seems likely to be slightly faster
and I don't see any real downside.
Fixed in the attached version(0004).
rhaas=# create table natch (a citext, b text) partition by hash (a);
ERROR: XX000: missing support function 2(16398,16398) in opfamily 16437
LOCATION: RelationBuildPartitionKey, relcache.c:954It shouldn't be possible to reach an elog() from SQL, and this is not
a friendly error message.
How about an error message in the attached patch(0004)?
1] /messages/by-id/CAAJ_b96AQBAxSQ2mxnTmx9zXh79GdP_dQWv0aupjcmz+jpiGjw@mail.gmail.com
Regards,
Amul
Attachments:
0001-provariadic-sanity-check.patchapplication/octet-stream; name=0001-provariadic-sanity-check.patchDownload
From e926fb9815a9d9d363cc075eccd71e5179c7fa1d Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 17:02:11 +0530
Subject: [PATCH 1/5] provariadic sanity check
---
src/test/regress/expected/type_sanity.out | 18 ++++++++++++++++++
src/test/regress/sql/type_sanity.sql | 16 ++++++++++++++++
2 files changed, 34 insertions(+)
diff --git a/src/test/regress/expected/type_sanity.out b/src/test/regress/expected/type_sanity.out
index 7b200baef8..c6440060dc 100644
--- a/src/test/regress/expected/type_sanity.out
+++ b/src/test/regress/expected/type_sanity.out
@@ -129,6 +129,24 @@ WHERE p1.typinput = p2.oid AND NOT
-----+---------+-----+---------
(0 rows)
+-- Check for type of the variadic array parameter's elements.
+-- provariadic should be ANYOID if the type of the last element is ANYOID,
+-- ANYELEMENTOID if the type of the last element is ANYARRAYOID, and otherwise
+-- the element type corresponding to the array type.
+SELECT oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
+FROM pg_proc
+WHERE provariadic != 0
+AND case proargtypes[array_length(proargtypes, 1)-1]
+ WHEN 2276 THEN 2276 -- any -> any
+ WHEN 2277 THEN 2283 -- anyarray -> anyelement
+ ELSE (SELECT t.oid
+ FROM pg_type t
+ WHERE t.typarray = proargtypes[array_length(proargtypes, 1)-1])
+ END != provariadic;
+ oid | provariadic | proargtypes
+-----+-------------+-------------
+(0 rows)
+
-- As of 8.0, this check finds refcursor, which is borrowing
-- other types' I/O routines
SELECT p1.oid, p1.typname, p2.oid, p2.proname
diff --git a/src/test/regress/sql/type_sanity.sql b/src/test/regress/sql/type_sanity.sql
index 4c65814008..428c2d324d 100644
--- a/src/test/regress/sql/type_sanity.sql
+++ b/src/test/regress/sql/type_sanity.sql
@@ -104,6 +104,22 @@ WHERE p1.typinput = p2.oid AND NOT
p2.proargtypes[1] = 'oid'::regtype AND
p2.proargtypes[2] = 'int4'::regtype));
+-- Check for type of the variadic array parameter's elements.
+-- provariadic should be ANYOID if the type of the last element is ANYOID,
+-- ANYELEMENTOID if the type of the last element is ANYARRAYOID, and otherwise
+-- the element type corresponding to the array type.
+
+SELECT oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
+FROM pg_proc
+WHERE provariadic != 0
+AND case proargtypes[array_length(proargtypes, 1)-1]
+ WHEN 2276 THEN 2276 -- any -> any
+ WHEN 2277 THEN 2283 -- anyarray -> anyelement
+ ELSE (SELECT t.oid
+ FROM pg_type t
+ WHERE t.typarray = proargtypes[array_length(proargtypes, 1)-1])
+ END != provariadic;
+
-- As of 8.0, this check finds refcursor, which is borrowing
-- other types' I/O routines
SELECT p1.oid, p1.typname, p2.oid, p2.proname
--
2.14.1
0002-Add-PG_GETARG_UINT64-macro.patchapplication/octet-stream; name=0002-Add-PG_GETARG_UINT64-macro.patchDownload
From f9c0229c3d4e13e2dac626ee92b5698a42a96ce7 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 15:34:10 +0530
Subject: [PATCH 2/5] Add PG_GETARG_UINT64 macro
---
src/include/fmgr.h | 1 +
1 file changed, 1 insertion(+)
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index a68ec91c68..96f20d3168 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -245,6 +245,7 @@ extern struct varlena *pg_detoast_datum_packed(struct varlena *datum);
#define PG_GETARG_FLOAT4(n) DatumGetFloat4(PG_GETARG_DATUM(n))
#define PG_GETARG_FLOAT8(n) DatumGetFloat8(PG_GETARG_DATUM(n))
#define PG_GETARG_INT64(n) DatumGetInt64(PG_GETARG_DATUM(n))
+#define PG_GETARG_UINT64(n) DatumGetUInt64(PG_GETARG_DATUM(n))
/* use this if you want the raw, possibly-toasted input datum: */
#define PG_GETARG_RAW_VARLENA_P(n) ((struct varlena *) PG_GETARG_POINTER(n))
/* use this if you want the input datum de-toasted: */
--
2.14.1
0003-partition_bounds_copy-code-refactoring-v1.patchapplication/octet-stream; name=0003-partition_bounds_copy-code-refactoring-v1.patchDownload
From f1535b84b4e36e4cdf38a6aebe4b71cfadd4768b Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:36:24 +0530
Subject: [PATCH 3/5] partition_bounds_copy code refactoring v1
---
src/backend/catalog/partition.c | 35 +++++++++++++++++++++++++++++++++--
1 file changed, 33 insertions(+), 2 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 66ec214e02..fc7b73a3bd 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -149,6 +149,7 @@ 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);
/*
* RelationBuildPartitionDesc
@@ -721,8 +722,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);
@@ -2894,3 +2894,34 @@ 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_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;
+}
--
2.14.1
0004-hash-partitioning_another_design-v26.patchapplication/octet-stream; name=0004-hash-partitioning_another_design-v26.patchDownload
From a309ef6b89b31a997e31a18ed997ea2db803f5cf Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 10:56:42 +0530
Subject: [PATCH 4/5] hash-partitioning_another_design-v26
v26:
Following changes w.r.t Robert's review comments[1]
- In pg_proc.h, argtype of satisfies_hash_func changed to ANYARRAYOID.
- Suggested document update.
- In RelationBuildPartitionKey, rewrote error message for missing support function,
1] https://postgr.es/m/CA+TgmoZ_UGXfq5ygeDDMdUSJ4J_VX7nFnjC6mfY6BgOJ3qZCmw@mail.gmail.com
v25:
Changes w.r.t Robert's review comments[1]
1] https://postgr.es/m/CA+TgmoYhB0+r--=bYkvexS7h+6P9GdoO8BN+vFWUuFxTUArZXQ@mail.gmail.com
v24:
- Added get_greatest_modulus() to as per Ashutosh's suggestion[1].
1] https://postgr.es/m/CAFjFpRexH08-rb2LzodCsEeVTbxRQiNuYgh7XfGJ1Gd4zg+AaQ@mail.gmail.com
v23 - v11 history:
Refer 0001-v23 patch @
https://postgr.es/m/CAAJ_b94A6NCGVkdQxuhLR8x0=20vaD9EH8mQ0Tj8X3M06Tp-Ow@mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 85 ++++-
src/backend/catalog/partition.c | 562 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 26 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 ++++
src/test/regress/expected/create_table.out | 83 ++++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1193 insertions(+), 104 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 03cbaa60ab..130e1b1145 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 41acda003f..3b19ea7131 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1431,6 +1431,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 4f7b741526..b476afd3f7 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="parameter">storage_parameter</replaceable> [= <replaceable class="parameter">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="parameter">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="parameter">storage_parameter</replaceable> [= <replaceable class="parameter">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="parameter">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="parameter">storage_parameter</replaceable> [= <replaceable class="parameter">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="parameter">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REMAINDER <replaceable class="parameter">numeric_literal</replaceable> )
<phrase><replaceable class="parameter">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -264,8 +266,9 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
must correspond to the partitioning method and partition key of the
parent table, and must not overlap with any existing partition of that
parent. The form with <literal>IN</literal> is used for list partitioning,
- while the form with <literal>FROM</literal> and <literal>TO</literal> is used for
- range partitioning.
+ the form with <literal>FROM</literal> and <literal>TO</literal> is used for
+ range partitioning, and the form with <literal>WITH</literal> is used for
+ hash partitioning.
</para>
<para>
@@ -363,6 +366,29 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,20 +512,28 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
of partitioning the table. The table thus created is called a
<firstterm>partitioned</firstterm> table. The parenthesized list of
columns or expressions forms the <firstterm>partition key</firstterm>
- for the table. When using range partitioning, the partition key can
- include multiple columns or expressions (up to 32, but this limit can be
- altered when building <productname>PostgreSQL</productname>), but for
+ for the table. When using range or hash partitioning, the partition key
+ can include multiple columns or expressions (up to 32, but this limit can
+ be altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression.
+ </para>
+
+ <para>
+ Range and list partitioning require a btree operator class, while hash
+ partitioning requires a hash operator class. If no operator class is
+ specified explicitly, the default operator class of the appropriate
+ type will be used; if no default operator class exists, an error will
+ be raised. When hash partitioning is used, the operator class used
+ must implement support function 2 (see <xref linkend="xindex-support">
+ for details).
</para>
<para>
@@ -1647,6 +1681,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1745,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index fc7b73a3bd..98ee9c1213 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -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);
@@ -150,6 +173,11 @@ static int partition_bound_bsearch(PartitionKey key,
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
@@ -175,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;
@@ -256,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;
@@ -485,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));
@@ -618,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];
@@ -656,49 +750,98 @@ 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 or
+ * same moduli with different number of partitions, 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -801,6 +944,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 +1397,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 +1772,105 @@ 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
+ * hash values computed for each column of the partition key using the extended
+ * hash function from the appropriate opclass.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_PARTITION_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2412,6 +2742,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 +2865,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 +3083,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],
@@ -2909,6 +3291,14 @@ get_partition_bound_num_indexes(PartitionBoundInfo 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;
@@ -2925,3 +3315,89 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ if (!PG_ARGISNULL(i + 2))
+ rowHash = hash_combine64(rowHash, PG_GETARG_UINT64(i + 2));
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3ab808715b..75fd87124d 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13271,7 +13271,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;
@@ -13341,10 +13343,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)
@@ -13504,25 +13508,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca909..cadd253ef1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 7a700018e7..2866fd7b4a 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 43d62062bc..291d1eeb46 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index ccb6a1f4ac..42c595dc03 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4c83a63f7d..92f5b86418 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..2af50419f3 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index b1e70a0d19..e87225c958 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8698,6 +8702,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..2a03c923a9 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,18 +943,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]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac0239d..8acc01a876 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.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.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..4d2840440b 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 0 0 f f f f f f i s 3 0 16 "23 23 2277" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 732e5d6788..97713934db 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index d7a084c5b7..eef3dd9625 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3399,6 +3399,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3596,6 +3597,59 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: updated partition constraint for default partition "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3607,12 +3661,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3709,6 +3768,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..84ea9df393 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,37 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 0);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 1);
+CREATE TABLE hpart_3 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 200, REMAINDER 2);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 3);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- previous modulus 50 is factor of 150 but this modulus is not factor of next modulus 200.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 150, REMAINDER 3);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: a hash-partitioned table may not have a default partition
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +561,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +661,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -694,6 +749,14 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Partitions: part_c_1_10 FOR VALUES FROM (1) TO (10)
+\d+ hpart_1
+ Table "public.hpart_1"
+ Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
+--------+---------+-----------+----------+---------+---------+--------------+-------------
+ a | integer | | | | plain | |
+Partition of: hash_parted FOR VALUES WITH (modulus 10, remainder 0)
+Partition constraint: pg_catalog.satisfies_hash_partition(10, 0, hashint4extended(a, '8816678312871386365'::bigint))
+
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
Table "public.part_c_1_10"
@@ -721,6 +784,14 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Number of partitions: 3 (Use \d+ to list them.)
+\d hash_parted
+ Table "public.hash_parted"
+ Column | Type | Collation | Nullable | Default
+--------+---------+-----------+----------+---------
+ a | integer | | |
+Partition key: HASH (a)
+Number of partitions: 3 (Use \d+ to list them.)
+
-- check that we get the expected partition constraints
CREATE TABLE range_parted4 (a int, b int, c int) PARTITION BY RANGE (abs(a), abs(b), c);
CREATE TABLE unbounded_range_part PARTITION OF range_parted4 FOR VALUES FROM (MINVALUE, MINVALUE, MINVALUE) TO (MAXVALUE, MAXVALUE, MAXVALUE);
@@ -771,6 +842,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..9d84ba4658 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..a4fe96112e 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 5);
+insert into hpart4 values (3, 4);
+-- fail
+update hpart1 set a = 3, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (3, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 339d25b5e5..8eb2c2ed82 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2139,6 +2139,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2332,6 +2333,62 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2343,12 +2400,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2421,6 +2482,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..b08b33d515 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,28 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 0);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 1);
+CREATE TABLE hpart_3 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 200, REMAINDER 2);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 3);
+-- previous modulus 50 is factor of 150 but this modulus is not factor of next modulus 200.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 150, REMAINDER 3);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +517,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +607,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -629,6 +666,7 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- Both partition bound and partition key in describe output
\d+ part_c
+\d+ hpart_1
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
@@ -638,6 +676,7 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- output could vary depending on the order in which partition oids are
-- returned.
\d parted
+\d hash_parted
-- check that we get the expected partition constraints
CREATE TABLE range_parted4 (a int, b int, c int) PARTITION BY RANGE (abs(a), abs(b), c);
@@ -654,6 +693,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..791817ba50 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..0c70d64a89 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 5);
+insert into hpart4 values (3, 4);
+
+-- fail
+update hpart1 set a = 3, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 7f0ae978c1..61aeb51c29 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1565,6 +1565,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
0005-Enable-partition-wise-join-support-v4.patchapplication/octet-stream; name=0005-Enable-partition-wise-join-support-v4.patchDownload
From 8e27aa011dc00786e51d6853561dd82027338461 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 11:01:15 +0530
Subject: [PATCH 5/5] Enable partition wise join support v4
v4:
-Comments changes in partition_bounds_copy() w.r.t Robert comments[1]
1] https://postgr.es/m/CA+TgmoZ_UGXfq5ygeDDMdUSJ4J_VX7nFnjC6mfY6BgOJ3qZCmw@mail.gmail.com
v3:
-In partition_bounds_copy, used natts variable instead
dim in the previous version.partition_bounds_copy and
added byval & typlen variable.
v2:
-Moved switch case from partition_bounds_copy() to 0001 patch.
v1:
-Few changes to enable partition wise join support for
the hash partitioned table.
-Added regression tests.
---
src/backend/catalog/partition.c | 29 ++++++++--
src/backend/optimizer/path/joinrels.c | 10 +++-
src/test/regress/expected/partition_join.out | 81 ++++++++++++++++++++++++++++
src/test/regress/sql/partition_join.sql | 32 +++++++++++
4 files changed, 146 insertions(+), 6 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 98ee9c1213..3cd3e30bac 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -891,15 +891,36 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
- dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
- 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);
}
}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c52de..5df861f9cf 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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. */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index adf6aedfa6..27ab8521f8 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1256,6 +1256,87 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
One-Time Filter: false
(14 rows)
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ QUERY PLAN
+--------------------------------------------------------------------------------------
+ Sort
+ Sort Key: t1.c, t3.c
+ -> HashAggregate
+ Group Key: t1.c, t2.c, t3.c
+ -> Result
+ -> Append
+ -> Hash Join
+ Hash Cond: (t1.c = t2.c)
+ -> Seq Scan on pht1_p1 t1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+ -> Seq Scan on pht2_p1 t2
+ -> Hash
+ -> Seq Scan on pht1_e_p1 t3
+ -> Hash Join
+ Hash Cond: (t1_1.c = t2_1.c)
+ -> Seq Scan on pht1_p2 t1_1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+ -> Seq Scan on pht2_p2 t2_1
+ -> Hash
+ -> Seq Scan on pht1_e_p2 t3_1
+ -> Hash Join
+ Hash Cond: (t1_2.c = t2_2.c)
+ -> Seq Scan on pht1_p3 t1_2
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+ -> Seq Scan on pht2_p3 t2_2
+ -> Hash
+ -> Seq Scan on pht1_e_p3 t3_2
+(33 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ avg | avg | avg | c | c | c
+----------------------+----------------------+-----------------------+------+------+-------
+ 24.0000000000000000 | 24.0000000000000000 | 48.0000000000000000 | 0000 | 0000 | A0000
+ 74.0000000000000000 | 75.0000000000000000 | 148.0000000000000000 | 0001 | 0001 | A0001
+ 124.0000000000000000 | 124.5000000000000000 | 248.0000000000000000 | 0002 | 0002 | A0002
+ 174.0000000000000000 | 174.0000000000000000 | 348.0000000000000000 | 0003 | 0003 | A0003
+ 224.0000000000000000 | 225.0000000000000000 | 448.0000000000000000 | 0004 | 0004 | A0004
+ 274.0000000000000000 | 274.5000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
+ 324.0000000000000000 | 324.0000000000000000 | 648.0000000000000000 | 0006 | 0006 | A0006
+ 374.0000000000000000 | 375.0000000000000000 | 748.0000000000000000 | 0007 | 0007 | A0007
+ 424.0000000000000000 | 424.5000000000000000 | 848.0000000000000000 | 0008 | 0008 | A0008
+ 474.0000000000000000 | 474.0000000000000000 | 948.0000000000000000 | 0009 | 0009 | A0009
+ 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
+ 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
+(12 rows)
+
--
-- multiple levels of partitioning
--
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 25abf2dc13..6efdf3c517 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -229,6 +229,38 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+
--
-- multiple levels of partitioning
--
--
2.14.1
On Mon, Oct 30, 2017 at 5:52 PM, amul sul <sulamul@gmail.com> wrote:
Actually, int4[] is also inappropriate type as we have started using a 64bit
hash function. We need something int8[] which is not available, so that I
have used ANYARRAYOID in the attached patch(0004).
I don't know why you think int8[] is not available.
rhaas=# select 'int8[]'::regtype;
regtype
----------
bigint[]
(1 row)
I wrote the following query
to detect problems of this type, and I think we might want to just go
ahead and add this to the regression test suite, verifying that it
returns no rows:select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
from pg_proc where provariadic != 0
and case proargtypes[array_length(proargtypes, 1)-1]
when 2276 then 2276 -- any -> any
when 2277 then 2283 -- anyarray -> anyelement
else (select t.oid from pg_type t where t.typarray =
proargtypes[array_length(proargtypes, 1)-1]) end
!= provariadic;Added in 0001 patch.
Committed.
One advantage of current implementation is that we can see which hash
function are used for the each partitioning column and also we don't need to
worry about user specified opclass and different input types.Something similar I've tried in my initial patch version[1], but I have missed
user specified opclass handling for each partitioning column. Do you want me
to handle opclass using RelabelType node? I am afraid that, that would make
the \d+ output more horrible than the current one if non-default opclass used.
Maybe we should just pass the OID of the partition (or both the
partition and the parent, so we can get the lock ordering right?)
instead.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 31, 2017 at 9:54 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Oct 30, 2017 at 5:52 PM, amul sul <sulamul@gmail.com> wrote:
Actually, int4[] is also inappropriate type as we have started using a 64bit
hash function. We need something int8[] which is not available, so that I
have used ANYARRAYOID in the attached patch(0004).I don't know why you think int8[] is not available.
rhaas=# select 'int8[]'::regtype;
regtype
----------
bigint[]
(1 row)
I missed _int8, was searching for INT8ARRAYOID in pg_type.h, my bad.
I wrote the following query
to detect problems of this type, and I think we might want to just go
ahead and add this to the regression test suite, verifying that it
returns no rows:select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
from pg_proc where provariadic != 0
and case proargtypes[array_length(proargtypes, 1)-1]
when 2276 then 2276 -- any -> any
when 2277 then 2283 -- anyarray -> anyelement
else (select t.oid from pg_type t where t.typarray =
proargtypes[array_length(proargtypes, 1)-1]) end
!= provariadic;Added in 0001 patch.
Committed.
Thanks !
One advantage of current implementation is that we can see which hash
function are used for the each partitioning column and also we don't need to
worry about user specified opclass and different input types.Something similar I've tried in my initial patch version[1], but I have missed
user specified opclass handling for each partitioning column. Do you want me
to handle opclass using RelabelType node? I am afraid that, that would make
the \d+ output more horrible than the current one if non-default opclass used.Maybe we should just pass the OID of the partition (or both the
partition and the parent, so we can get the lock ordering right?)
instead.
Okay, will try this.
Regards,
Amul
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Oct 31, 2017 at 10:17 AM, amul sul <sulamul@gmail.com> wrote:
On Tue, Oct 31, 2017 at 9:54 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Oct 30, 2017 at 5:52 PM, amul sul <sulamul@gmail.com> wrote:
Actually, int4[] is also inappropriate type as we have started using a 64bit
hash function. We need something int8[] which is not available, so that I
have used ANYARRAYOID in the attached patch(0004).I don't know why you think int8[] is not available.
rhaas=# select 'int8[]'::regtype;
regtype
----------
bigint[]
(1 row)I missed _int8, was searching for INT8ARRAYOID in pg_type.h, my bad.
Fixed in the 0003 patch.
[....]
Something similar I've tried in my initial patch version[1], but I have missed
user specified opclass handling for each partitioning column. Do you want me
to handle opclass using RelabelType node? I am afraid that, that would make
the \d+ output more horrible than the current one if non-default opclass used.Maybe we should just pass the OID of the partition (or both the
partition and the parent, so we can get the lock ordering right?)
instead.Okay, will try this.
In 0005, I rewrote satisfies_hash_partition, to accept parent id, modulus and
remainder as before, and the column values directly. This function opens parent
relation to get its PartitionKey which has extended hash function information in
a partsupfunc array, using this it will calculates a hash for the partition key.
Also, it will copy this partsupfunc array into function memory context so that
we don't need to open parent relation again and again in the subsequent function
call to get extended hash functions information (e.g. bulk insert).
In \d+ partition constraint will be :
satisfies_hash_partition('16384'::oid, 2, 0, a, b)
where 16384 is parent relid, 2 is modulus, 0 is remainder and 'a' &
'b' are partition
column.
In the earlier version partition constraint was (i.e. without 0005 patch):
satisfies_hash_partition(2, 0,
hashint4extended(a,'8816678312871386365'::bigint),
hashtextextended(b, '8816678312871386365'::bigint))
I did small performance test using a copy command to load 100,000,000 records
and a separate insert command for each record to load 2,00,000 records and
result are as follow:
+---------+-----------------+--------------------+
| Command | With 0005 patch | Without 0005 patch |
+---------+-----------------+--------------------+
| COPY | 63.719 seconds | 64.925 seconds |
+---------+-----------------+--------------------+
| INSERT | 179.21 seconds | 174.89 seconds |
+---------+-----------------+--------------------+
Although partition constraints become more simple, there isn't any performance
gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
copied extended hash function info from the partition key, what if parent is
changed while we are using it? Do we need to keep lock on parent until commit in
satisfies_hash_partition?
Regards,
Amul
Attachments:
0003-hash-partitioning_another_design-v27.patchapplication/octet-stream; name=0003-hash-partitioning_another_design-v27.patchDownload
From 388394a747316fceb37392ad918280c2460fbce9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 10:56:42 +0530
Subject: [PATCH 3/5] hash-partitioning_another_design-v27
v27:
- In pg_proc.h, argtype of satisfies_hash_func changed to int8[].
v26:
Following changes w.r.t Robert's review comments[1]
- In pg_proc.h, argtype of satisfies_hash_func changed to ANYARRAYOID.
- Suggested document update.
- In RelationBuildPartitionKey, rewrote error message for missing support function,
1] https://postgr.es/m/CA+TgmoZ_UGXfq5ygeDDMdUSJ4J_VX7nFnjC6mfY6BgOJ3qZCmw@mail.gmail.com
v25:
Changes w.r.t Robert's review comments[1]
1] https://postgr.es/m/CA+TgmoYhB0+r--=bYkvexS7h+6P9GdoO8BN+vFWUuFxTUArZXQ@mail.gmail.com
v24:
- Added get_greatest_modulus() to as per Ashutosh's suggestion[1].
1] https://postgr.es/m/CAFjFpRexH08-rb2LzodCsEeVTbxRQiNuYgh7XfGJ1Gd4zg+AaQ@mail.gmail.com
v23 - v11 history:
Refer 0001-v23 patch @
https://postgr.es/m/CAAJ_b94A6NCGVkdQxuhLR8x0=20vaD9EH8mQ0Tj8X3M06Tp-Ow@mail.gmail.com
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 85 ++++-
src/backend/catalog/partition.c | 562 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 48 ++-
src/backend/nodes/copyfuncs.c | 2 +
src/backend/nodes/equalfuncs.c | 2 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 2 +
src/backend/parser/gram.y | 76 +++-
src/backend/parser/parse_utilcmd.c | 27 +-
src/backend/utils/adt/ruleutils.c | 15 +-
src/backend/utils/cache/relcache.c | 26 +-
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/partition.h | 3 +
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 62 ++++
src/test/regress/expected/create_table.out | 83 ++++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 64 ++++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
src/tools/pgindent/typedefs.list | 1 +
26 files changed, 1193 insertions(+), 104 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 03cbaa60ab..130e1b1145 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2875,6 +2875,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
</para>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying a modulus and a remainder for each
+ partition. Each partition will hold the rows for which the hash value of
+ the partition key divided by the specified modulus will produce the specified
+ remainder. Refer to <xref linkend="sql-createtable-partition">
+ for additional clarification on modulus and remainder.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
If your application needs to use other forms of partitioning not listed
@@ -2901,9 +2915,8 @@ VALUES ('Albany', NULL, NULL, 'NY');
All rows inserted into a partitioned table will be routed to one of the
<firstterm>partitions</firstterm> based on the value of the partition
key. Each partition has a subset of the data defined by its
- <firstterm>partition bounds</firstterm>. Currently supported
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ <firstterm>partition bounds</firstterm>. The currently supported
+ partitioning methods are range, list, and hash.
</para>
<para>
@@ -3328,11 +3341,11 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
<listitem>
<para>
- Declarative partitioning only supports list and range partitioning,
- whereas table inheritance allows data to be divided in a manner of
- the user's choosing. (Note, however, that if constraint exclusion is
- unable to prune partitions effectively, query performance will be very
- poor.)
+ Declarative partitioning only supports range, list and hash
+ partitioning, whereas table inheritance allows data to be divided in a
+ manner of the user's choosing. (Note, however, that if constraint
+ exclusion is unable to prune partitions effectively, query performance
+ will be very poor.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 41acda003f..3b19ea7131 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -1431,6 +1431,13 @@ ALTER TABLE cities
ATTACH PARTITION cities_partdef DEFAULT;
</programlisting></para>
+ <para>
+ Attach a partition to hash partitioned table:
+<programlisting>
+ALTER TABLE orders
+ ATTACH PARTITION orders_p4 FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Detach a partition from partitioned table:
<programlisting>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 4f7b741526..b476afd3f7 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -28,7 +28,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
[, ... ]
] )
[ INHERITS ( <replaceable>parent_table</replaceable> [, ... ] ) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="parameter">storage_parameter</replaceable> [= <replaceable class="parameter">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="parameter">tablespace_name</replaceable> ]
@@ -39,7 +39,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ]
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="parameter">storage_parameter</replaceable> [= <replaceable class="parameter">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="parameter">tablespace_name</replaceable> ]
@@ -50,7 +50,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
| <replaceable>table_constraint</replaceable> }
[, ... ]
) ] { FOR VALUES <replaceable class="parameter">partition_bound_spec</replaceable> | DEFAULT }
-[ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
+[ PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
[ WITH ( <replaceable class="parameter">storage_parameter</replaceable> [= <replaceable class="parameter">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
[ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
[ TABLESPACE <replaceable class="parameter">tablespace_name</replaceable> ]
@@ -88,7 +88,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
IN ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | NULL } [, ...] ) |
FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
- TO ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] )
+ TO ( { <replaceable class="parameter">numeric_literal</replaceable> | <replaceable class="parameter">string_literal</replaceable> | MINVALUE | MAXVALUE } [, ...] ) |
+WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REMAINDER <replaceable class="parameter">numeric_literal</replaceable> )
<phrase><replaceable class="parameter">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -256,7 +257,8 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>.
+ using <literal>DEFAULT</literal>. Hash partitioned tables do not support
+ a default partition.
</para>
<para>
@@ -264,8 +266,9 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
must correspond to the partitioning method and partition key of the
parent table, and must not overlap with any existing partition of that
parent. The form with <literal>IN</literal> is used for list partitioning,
- while the form with <literal>FROM</literal> and <literal>TO</literal> is used for
- range partitioning.
+ the form with <literal>FROM</literal> and <literal>TO</literal> is used for
+ range partitioning, and the form with <literal>WITH</literal> is used for
+ hash partitioning.
</para>
<para>
@@ -363,6 +366,29 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
partition.
</para>
+ <para>
+ When creating a hash partition, a modulus and remainder must be specified.
+ The modulus must be a positive integer, and the remainder must be a
+ non-negative integer less than the modulus. Typically, when initially
+ setting up a hash-partitioned table, you should choose a modulus equal to
+ the number of partitions and assign every table the same modulus and a
+ different remainder (see examples, below). However, it is not required
+ that every partition have the same modulus, only that every modulus which
+ occurs among the partitions of a hash-partitioned table is a factor of the
+ next larger modulus. This allows the number of partitions to be increased
+ incrementally without needing to move all the data at once. For example,
+ suppose you have a hash-partitioned table with 8 partitions, each of which
+ has modulus 8, but find it necessary to increase the number of partitions
+ to 16. You can detach one of the modulus-8 partitions, create two new
+ modulus-16 partitions covering the same portion of the key space (one with
+ a remainder equal to the remainder of the detached partition, and the
+ other with a remainder equal to that value plus 8), and repopulate them
+ with data. You can then repeat this -- perhaps at a later time -- for
+ each modulus-8 partition until none remain. While this may still involve
+ a large amount of data movement at each step, it is still better than
+ having to create a whole new table and move all the data at once.
+ </para>
+
<para>
A partition must have the same column names and types as the partitioned
table to which it belongs. If the parent is specified <literal>WITH
@@ -486,20 +512,28 @@ FROM ( { <replaceable class="parameter">numeric_literal</replaceable> | <replace
</varlistentry>
<varlistentry>
- <term><literal>PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
+ <term><literal>PARTITION BY { RANGE | LIST | HASH } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ <replaceable class="parameter">opclass</replaceable> ] [, ...] ) </literal></term>
<listitem>
<para>
The optional <literal>PARTITION BY</literal> clause specifies a strategy
of partitioning the table. The table thus created is called a
<firstterm>partitioned</firstterm> table. The parenthesized list of
columns or expressions forms the <firstterm>partition key</firstterm>
- for the table. When using range partitioning, the partition key can
- include multiple columns or expressions (up to 32, but this limit can be
- altered when building <productname>PostgreSQL</productname>), but for
+ for the table. When using range or hash partitioning, the partition key
+ can include multiple columns or expressions (up to 32, but this limit can
+ be altered when building <productname>PostgreSQL</productname>), but for
list partitioning, the partition key must consist of a single column or
- expression. If no B-tree operator class is specified when creating a
- partitioned table, the default B-tree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression.
+ </para>
+
+ <para>
+ Range and list partitioning require a btree operator class, while hash
+ partitioning requires a hash operator class. If no operator class is
+ specified explicitly, the default operator class of the appropriate
+ type will be used; if no default operator class exists, an error will
+ be raised. When hash partitioning is used, the operator class used
+ must implement support function 2 (see <xref linkend="xindex-support">
+ for details).
</para>
<para>
@@ -1647,6 +1681,16 @@ CREATE TABLE cities (
name text not null,
population bigint
) PARTITION BY LIST (left(lower(name), 1));
+</programlisting></para>
+
+ <para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE orders (
+ order_id bigint not null,
+ cust_id bigint not null,
+ status text
+) PARTITION BY HASH (order_id);
</programlisting></para>
<para>
@@ -1701,6 +1745,19 @@ CREATE TABLE cities_ab_10000_to_100000
PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
</programlisting></para>
+ <para>
+ Create partitions of a hash partitioned table:
+<programlisting>
+CREATE TABLE orders_p1 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE orders_p2 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+CREATE TABLE orders_p3 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE orders_p4 PARTITION OF orders
+ FOR VALUES WITH (MODULUS 4, REMAINDER 3);
+</programlisting></para>
+
<para>
Create a default partition:
<programlisting>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index e900899cc6..b22fa95478 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -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);
@@ -150,6 +173,11 @@ static int partition_bound_bsearch(PartitionKey key,
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
@@ -175,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;
@@ -256,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;
@@ -485,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));
@@ -618,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];
@@ -656,49 +750,98 @@ 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 or
+ * same moduli with different number of partitions, 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)
+ int j;
+
+ for (j = 0; j < partnatts; j++)
{
- /* The different kinds of bound all differ from each other */
- if (b1->kind[i][j] != b2->kind[i][j])
- return false;
+ /* 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;
+ /*
+ * 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;
}
@@ -801,6 +944,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 +1397,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 +1772,105 @@ 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
+ * hash values computed for each column of the partition key using the extended
+ * hash function from the appropriate opclass.
+ */
+static List *
+get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ Node *hashSeedConst;
+ List *args;
+ ListCell *partexprs_item;
+ int i;
+
+ /* Default hash partition is not supported */
+ Assert(!spec->is_default);
+
+ /* Fixed arguments. */
+ 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);
+
+ hashSeedConst = (Node *) makeConst(INT8OID,
+ -1,
+ InvalidOid,
+ sizeof(uint64),
+ UInt64GetDatum(HASH_PARTITION_SEED),
+ false,
+ FLOAT8PASSBYVAL);
+
+ args = list_make2(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
+ {
+ if (partexprs_item == NULL)
+ elog(ERROR, "wrong number of partition key expressions");
+
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ /* Form hash_fn(keyCol) expression */
+ fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_func_rettype(key->partsupfunc[i].fn_oid),
+ list_make2(keyCol, hashSeedConst),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ args = lappend(args, fexpr);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
/*
* get_qual_for_list
*
@@ -2412,6 +2742,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 +2865,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 +3083,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],
@@ -2909,6 +3291,14 @@ get_partition_bound_num_indexes(PartitionBoundInfo 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;
@@ -2925,3 +3315,89 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
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)
+{
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ uint64 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+
+ /*
+ * Partition key attribute's hash values start from third argument of
+ * function.
+ */
+ if (!PG_ARGISNULL(i + 2))
+ rowHash = hash_combine64(rowHash, PG_GETARG_UINT64(i + 2));
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3ab808715b..75fd87124d 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -470,7 +470,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,
@@ -893,7 +893,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);
@@ -13271,7 +13271,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;
@@ -13341,10 +13343,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)
@@ -13504,25 +13508,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++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca909..cadd253ef1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -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);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 7a700018e7..2866fd7b4a 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -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);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 43d62062bc..291d1eeb46 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -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);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index ccb6a1f4ac..42c595dc03 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -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);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 4c83a63f7d..92f5b86418 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -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); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 27e568fc62..2af50419f3 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -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("modulus for hash partition must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index b1e70a0d19..e87225c958 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1550,7 +1550,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)
@@ -1654,6 +1654,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");
@@ -8698,6 +8702,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);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index b8e37809b0..2a03c923a9 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -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"
@@ -838,6 +839,7 @@ RelationBuildPartitionKey(Relation relation)
Datum datum;
MemoryContext partkeycxt,
oldcxt;
+ int16 procnum;
tuple = SearchSysCache1(PARTRELID,
ObjectIdGetDatum(RelationGetRelid(relation)));
@@ -917,6 +919,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);
@@ -937,18 +943,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]);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a09c49d6cf..b3e3799c13 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2055,7 +2055,7 @@ psql_completion(const char *text, int start, int end)
else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
else if (TailMatches2("FOR", "VALUES"))
- COMPLETE_WITH_LIST2("FROM (", "IN (");
+ COMPLETE_WITH_LIST3("FROM (", "IN (", "WITH (");
/*
* If we have ALTER TABLE <foo> DETACH PARTITION, provide a list of
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac0239d..8acc01a876 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -19,6 +19,9 @@
#include "parser/parse_node.h"
#include "utils/rel.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.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 93c031aad7..87acdc0ab3 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5522,6 +5522,10 @@ DESCR("list files in the log directory");
DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t v s 0 0 2249 "" "{25,20,1184}" "{o,o,o}" "{name,size,modification}" _null_ _null_ pg_ls_waldir _null_ _null_ _null_ ));
DESCR("list of files in the WAL directory");
+/* hash partitioning constraint function */
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 20 0 f f f f f f i s 3 0 16 "23 23 1016" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 732e5d6788..97713934db 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -777,12 +777,14 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('hash', 'list' or
+ * 'range') */
List *partParams; /* List of PartitionElems */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
/* Internal codes for partitioning strategies */
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -799,6 +801,10 @@ typedef struct PartitionBoundSpec
char strategy; /* see PARTITION_STRATEGY codes above */
bool is_default; /* is it a default partition bound? */
+ /* Partitioning info for HASH strategy: */
+ int modulus;
+ int remainder;
+
/* Partitioning info for LIST strategy: */
List *listdatums; /* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index d7a084c5b7..eef3dd9625 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3399,6 +3399,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
ERROR: partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -3596,6 +3597,59 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: updated partition constraint for default partition "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3607,12 +3661,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
ERROR: relation "part_4" does not exist
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
+ERROR: relation "hpart_4" does not exist
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ERROR: relation "not_a_part" is not a partition of relation "list_parted2"
ALTER TABLE list_parted2 DETACH PARTITION part_1;
ERROR: relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted"
+DROP TABLE not_a_part;
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3709,6 +3768,9 @@ ERROR: cannot alter type of column named in partition key
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 60ab28a96a..5019d1caaf 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-ERROR: unrecognized partitioning strategy "hash"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -467,6 +462,11 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
ERROR: invalid bound specification for a list partition
LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
^
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a list partition
+LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODU...
+ ^
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
@@ -509,6 +509,11 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
ERROR: invalid bound specification for a range partition
LINE 1: ...BLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
^
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -518,6 +523,37 @@ ERROR: TO must specify exactly one value per partitioning column
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
ERROR: cannot specify NULL in range bound
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+ERROR: invalid bound specification for a range partition
+LINE 1: ...LE fail_part PARTITION OF range_parted FOR VALUES WITH (MODU...
+ ^
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 0);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 1);
+CREATE TABLE hpart_3 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 200, REMAINDER 2);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 3);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- previous modulus 50 is factor of 150 but this modulus is not factor of next modulus 200.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 150, REMAINDER 3);
+ERROR: every hash partition modulus must be a factor of the next larger modulus
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+ ^
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+ ^
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+ERROR: a hash-partitioned table may not have a default partition
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -525,6 +561,8 @@ CREATE TABLE unparted (
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
ERROR: "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: "unparted" is not partitioned
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
CREATE TEMP TABLE temp_parted (
@@ -623,6 +661,23 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
ERROR: partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ERROR: partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ERROR: modulus for hash partition must be a positive integer
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ERROR: modulus for hash partition must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -694,6 +749,14 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Partitions: part_c_1_10 FOR VALUES FROM (1) TO (10)
+\d+ hpart_1
+ Table "public.hpart_1"
+ Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
+--------+---------+-----------+----------+---------+---------+--------------+-------------
+ a | integer | | | | plain | |
+Partition of: hash_parted FOR VALUES WITH (modulus 10, remainder 0)
+Partition constraint: satisfies_hash_partition(10, 0, hashint4extended(a, '8816678312871386365'::bigint))
+
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
Table "public.part_c_1_10"
@@ -721,6 +784,14 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Number of partitions: 3 (Use \d+ to list them.)
+\d hash_parted
+ Table "public.hash_parted"
+ Column | Type | Collation | Nullable | Default
+--------+---------+-----------+----------+---------
+ a | integer | | |
+Partition key: HASH (a)
+Number of partitions: 3 (Use \d+ to list them.)
+
-- check that we get the expected partition constraints
CREATE TABLE range_parted4 (a int, b int, c int) PARTITION BY RANGE (abs(a), abs(b), c);
CREATE TABLE unbounded_range_part PARTITION OF range_parted4 FOR VALUES FROM (MINVALUE, MINVALUE, MINVALUE) TO (MAXVALUE, MAXVALUE, MAXVALUE);
@@ -771,6 +842,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND (c IS N
DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
COMMENT ON TABLE parted_col_comment IS 'Am partitioned table';
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b715619313..9d84ba4658 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -382,8 +382,54 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
part_null | | 1 | 1
(9 rows)
+-- direct partition inserts should check hash partition bound constraint
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+insert into hash_parted values(generate_series(1,10));
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+ERROR: new row for relation "hpart0" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+ part | a | remainder = a % 4
+--------+----+-------------------
+ hpart0 | 4 | 0
+ hpart0 | 8 | 0
+ hpart0 | 12 | 0
+ hpart0 | 16 | 0
+ hpart1 | 1 | 1
+ hpart1 | 5 | 1
+ hpart1 | 9 | 1
+ hpart2 | 2 | 2
+ hpart2 | 6 | 2
+ hpart2 | 10 | 2
+ hpart3 | 3 | 3
+ hpart3 | 7 | 3
+ hpart3 | 11 | 3
+(13 rows)
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
create table list_parted (a int) partition by list (a);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index cef70b1a1e..a4fe96112e 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -250,6 +250,35 @@ ERROR: new row for relation "list_default" violates partition constraint
DETAIL: Failing row contains (a, 10).
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 5);
+insert into hpart4 values (3, 4);
+-- fail
+update hpart1 set a = 3, b=4 where a = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (3, 4).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (1, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 339d25b5e5..8eb2c2ed82 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2139,6 +2139,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
-- check that the new partition won't overlap with an existing partition
CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
-- check that an existing table can be attached as a default partition
CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
@@ -2332,6 +2333,62 @@ CREATE TABLE quuux1 PARTITION OF quuux FOR VALUES IN (1);
CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
DROP TABLE quuux;
+-- check validation when attaching hash partitions
+
+-- The default hash functions as they exist today aren't portable, they can
+-- return different results on different machines. Depending upon how the
+-- values are hashed, the row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create a non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4, seed int8) RETURNS int8 AS
+$$ BEGIN RETURN (a + 1 + seed); END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 2 dummy_hashint4(int4, int8);
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int,
+ b int
+) PARTITION BY HASH (a custom_opclass);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 4, REMAINDER 0);
+CREATE TABLE fail_part (LIKE hpart_1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted);
+INSERT INTO hpart_2 VALUES (3, 0);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_2 FOR VALUES WITH (MODULUS 4, REMAINDER 1);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('1', '2', '3');
+INSERT INTO hpart_5_a (a, b) VALUES (7, 1);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_5_a;
+ALTER TABLE hash_parted ATTACH PARTITION hpart_5 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2343,12 +2400,16 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted DETACH PARTITION hpart_4;
-- check that the partition being detached is actually a partition of the parent
CREATE TABLE not_a_part (a int);
ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
ALTER TABLE list_parted2 DETACH PARTITION part_1;
+ALTER TABLE hash_parted DETACH PARTITION not_a_part;
+DROP TABLE not_a_part;
+
-- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
-- attislocal/conislocal is set to true
ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2421,6 +2482,9 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
DROP TABLE fail_def_part;
+DROP TABLE hash_parted;
+DROP OPERATOR CLASS custom_opclass USING HASH;
+DROP FUNCTION dummy_hashint4(a int4, seed int8);
-- more tests for certain multi-level partitioning scenarios
create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index df6a6d7326..b08b33d515 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
- a int
-) PARTITION BY HASH (a);
-
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -446,6 +441,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
-- trying to specify range for list partitioned table
CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
+-- trying to specify modulus and remainder for list partitioned table
+CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- check default partition cannot be created more than once
CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
@@ -481,6 +478,8 @@ CREATE TABLE range_parted (
-- trying to specify list for range partitioned table
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES IN ('a');
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
-- each of start and end bounds must have same number of values as the
-- length of the partition key
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a', 1) TO ('z');
@@ -489,6 +488,28 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
-- cannot specify null values in range bounds
CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (maxvalue);
+-- trying to specify modulus and remainder for range partitioned table
+CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES WITH (MODULUS 10, REMAINDER 1);
+
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+ a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 10, REMAINDER 0);
+CREATE TABLE hpart_2 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 50, REMAINDER 1);
+CREATE TABLE hpart_3 PARTITION OF hash_parted FOR VALUES WITH (MODULUS 200, REMAINDER 2);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 25, REMAINDER 3);
+-- previous modulus 50 is factor of 150 but this modulus is not factor of next modulus 200.
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES WITH (MODULUS 150, REMAINDER 3);
+-- trying to specify range for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+-- trying to specify list value for the hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES IN (1000);
+
+-- trying to create default partition for the hash partitioned table
+CREATE TABLE fail_default_part PARTITION OF hash_parted DEFAULT;
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -496,6 +517,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted FOR VALUES WITH (MODULUS 2, REMAINDER 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -585,6 +607,21 @@ CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO (1, maxvalue);
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+ a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REMAINDER 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -629,6 +666,7 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- Both partition bound and partition key in describe output
\d+ part_c
+\d+ hpart_1
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
@@ -638,6 +676,7 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- output could vary depending on the order in which partition oids are
-- returned.
\d parted
+\d hash_parted
-- check that we get the expected partition constraints
CREATE TABLE range_parted4 (a int, b int, c int) PARTITION BY RANGE (abs(a), abs(b), c);
@@ -654,6 +693,8 @@ DROP TABLE range_parted4;
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted;
+DROP TABLE hash_parted2;
-- comments on partitioned tables columns
CREATE TABLE parted_col_comment (a int, b text) PARTITION BY LIST (a);
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index d741514414..791817ba50 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -222,8 +222,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
insert into list_parted (b) values (1);
select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
+-- direct partition inserts should check hash partition bound constraint
+
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3);
+
+insert into hash_parted values(generate_series(1,10));
+
+-- direct insert of values divisible by 4 - ok;
+insert into hpart0 values(12),(16);
+-- fail;
+insert into hpart0 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition
+insert into hpart3 values(11);
+
+-- view data
+select tableoid::regclass as part, a, a%4 as "remainder = a % 4"
+from hash_parted order by part;
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
-- test that a default partition added as the first partition accepts any value
-- including null
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 66d1feca10..0c70d64a89 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -148,6 +148,34 @@ update list_default set a = 'a' where a = 'd';
-- ok
update list_default set a = 'x' where a = 'd';
+-- create custom operator class and hash function, for the same reason
+-- explained in alter_table.sql
+create or replace function dummy_hashint4(a int4, seed int8) returns int8 as
+$$ begin return (a + seed); end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 2 dummy_hashint4(int4, int8);
+
+create table hash_parted (
+ a int,
+ b int
+) partition by hash (a custom_opclass, b custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted for values with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 4);
+insert into hpart1 values (1, 1);
+insert into hpart2 values (2, 5);
+insert into hpart4 values (3, 4);
+
+-- fail
+update hpart1 set a = 3, b=4 where a = 1;
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
-- cleanup
drop table range_parted;
drop table list_parted;
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function dummy_hashint4(a int4, seed int8);
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 7f0ae978c1..61aeb51c29 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1565,6 +1565,7 @@ PartitionDispatch
PartitionDispatchData
PartitionElem
PartitionKey
+PartitionHashBound
PartitionListValue
PartitionRangeBound
PartitionRangeDatum
--
2.14.1
0004-Enable-partition-wise-join-support-v4.patchapplication/octet-stream; name=0004-Enable-partition-wise-join-support-v4.patchDownload
From 636ab990634bdb3c4537c38cd5b278fdf1ebe752 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 11:01:15 +0530
Subject: [PATCH 4/5] Enable partition wise join support v4
v4:
-Comments changes in partition_bounds_copy() w.r.t Robert comments[1]
1] https://postgr.es/m/CA+TgmoZ_UGXfq5ygeDDMdUSJ4J_VX7nFnjC6mfY6BgOJ3qZCmw@mail.gmail.com
v3:
-In partition_bounds_copy, used natts variable instead
dim in the previous version.partition_bounds_copy and
added byval & typlen variable.
v2:
-Moved switch case from partition_bounds_copy() to 0001 patch.
v1:
-Few changes to enable partition wise join support for
the hash partitioned table.
-Added regression tests.
---
src/backend/catalog/partition.c | 29 ++++++++--
src/backend/optimizer/path/joinrels.c | 10 +++-
src/test/regress/expected/partition_join.out | 81 ++++++++++++++++++++++++++++
src/test/regress/sql/partition_join.sql | 32 +++++++++++
4 files changed, 146 insertions(+), 6 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b22fa95478..a900ede3cb 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -891,15 +891,36 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
int j;
- dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
- 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);
}
}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c52de..5df861f9cf 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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. */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index adf6aedfa6..27ab8521f8 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1256,6 +1256,87 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
One-Time Filter: false
(14 rows)
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ QUERY PLAN
+--------------------------------------------------------------------------------------
+ Sort
+ Sort Key: t1.c, t3.c
+ -> HashAggregate
+ Group Key: t1.c, t2.c, t3.c
+ -> Result
+ -> Append
+ -> Hash Join
+ Hash Cond: (t1.c = t2.c)
+ -> Seq Scan on pht1_p1 t1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+ -> Seq Scan on pht2_p1 t2
+ -> Hash
+ -> Seq Scan on pht1_e_p1 t3
+ -> Hash Join
+ Hash Cond: (t1_1.c = t2_1.c)
+ -> Seq Scan on pht1_p2 t1_1
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+ -> Seq Scan on pht2_p2 t2_1
+ -> Hash
+ -> Seq Scan on pht1_e_p2 t3_1
+ -> Hash Join
+ Hash Cond: (t1_2.c = t2_2.c)
+ -> Seq Scan on pht1_p3 t1_2
+ -> Hash
+ -> Hash Join
+ Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+ -> Seq Scan on pht2_p3 t2_2
+ -> Hash
+ -> Seq Scan on pht1_e_p3 t3_2
+(33 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+ avg | avg | avg | c | c | c
+----------------------+----------------------+-----------------------+------+------+-------
+ 24.0000000000000000 | 24.0000000000000000 | 48.0000000000000000 | 0000 | 0000 | A0000
+ 74.0000000000000000 | 75.0000000000000000 | 148.0000000000000000 | 0001 | 0001 | A0001
+ 124.0000000000000000 | 124.5000000000000000 | 248.0000000000000000 | 0002 | 0002 | A0002
+ 174.0000000000000000 | 174.0000000000000000 | 348.0000000000000000 | 0003 | 0003 | A0003
+ 224.0000000000000000 | 225.0000000000000000 | 448.0000000000000000 | 0004 | 0004 | A0004
+ 274.0000000000000000 | 274.5000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
+ 324.0000000000000000 | 324.0000000000000000 | 648.0000000000000000 | 0006 | 0006 | A0006
+ 374.0000000000000000 | 375.0000000000000000 | 748.0000000000000000 | 0007 | 0007 | A0007
+ 424.0000000000000000 | 424.5000000000000000 | 848.0000000000000000 | 0008 | 0008 | A0008
+ 474.0000000000000000 | 474.0000000000000000 | 948.0000000000000000 | 0009 | 0009 | A0009
+ 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
+ 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
+(12 rows)
+
--
-- multiple levels of partitioning
--
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 25abf2dc13..6efdf3c517 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -229,6 +229,38 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
+--
+-- tests for hash partitioned tables.
+--
+CREATE TABLE pht1 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht1_p1 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_p2 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_p3 PARTITION OF pht1 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1;
+
+CREATE TABLE pht2 (a int, b int, c text) PARTITION BY HASH(c);
+CREATE TABLE pht2_p1 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht2_p2 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht2_p3 PARTITION OF pht2 FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE pht2;
+
+--
+-- hash partitioned by expression
+--
+CREATE TABLE pht1_e (a int, b int, c text) PARTITION BY HASH(ltrim(c, 'A'));
+CREATE TABLE pht1_e_p1 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 0);
+CREATE TABLE pht1_e_p2 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 1);
+CREATE TABLE pht1_e_p3 PARTITION OF pht1_e FOR VALUES WITH (MODULUS 3, REMAINDER 2);
+INSERT INTO pht1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE pht1_e;
+
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+
--
-- multiple levels of partitioning
--
--
2.14.1
0005-satisfies_hash_partition-signature-change-WIP.patchapplication/octet-stream; name=0005-satisfies_hash_partition-signature-change-WIP.patchDownload
From 479b317df0d6096670490eda0a0a13c7810dc3c6 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Wed, 1 Nov 2017 11:44:11 +0530
Subject: [PATCH 5/5] satisfies_hash_partition signature change WIP.
In this satisfies_hash_partition accepts parent relation id,
modulus, remainder and partition key value.
For the batch insert operation we'll open parent relation
and get the extended hash function info from the partition
key and stored in fn_extra.
TODO:
1. Do we need to hold a lock on parent until commit?
2. What is input type if different from the partition key?
3. What is parent relation is change when we are using store function
info?
---
src/backend/catalog/partition.c | 106 ++++++++++++++++++++---------
src/include/catalog/pg_proc.h | 2 +-
src/test/regress/expected/create_table.out | 8 ---
src/test/regress/sql/create_table.sql | 1 -
4 files changed, 73 insertions(+), 44 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index a900ede3cb..b8e800db7e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1811,17 +1811,22 @@ get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
{
PartitionKey key = RelationGetPartitionKey(parent);
FuncExpr *fexpr;
+ Node *relidConst;
Node *modulusConst;
Node *remainderConst;
- Node *hashSeedConst;
List *args;
ListCell *partexprs_item;
int i;
- /* Default hash partition is not supported */
- Assert(!spec->is_default);
-
/* Fixed arguments. */
+ relidConst = (Node *) makeConst(OIDOID,
+ -1,
+ InvalidOid,
+ sizeof(Oid),
+ ObjectIdGetDatum(RelationGetRelid(parent)),
+ false,
+ true);
+
modulusConst = (Node *) makeConst(INT4OID,
-1,
InvalidOid,
@@ -1838,15 +1843,7 @@ get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
false,
true);
- hashSeedConst = (Node *) makeConst(INT8OID,
- -1,
- InvalidOid,
- sizeof(uint64),
- UInt64GetDatum(HASH_PARTITION_SEED),
- false,
- FLOAT8PASSBYVAL);
-
- args = list_make2(modulusConst, remainderConst);
+ args = list_make3(relidConst, modulusConst, remainderConst);
partexprs_item = list_head(key->partexprs);
/* Add an argument for each key column. */
@@ -1856,30 +1853,21 @@ get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
/* 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
{
- if (partexprs_item == NULL)
- elog(ERROR, "wrong number of partition key expressions");
-
keyCol = (Node *) copyObject(lfirst(partexprs_item));
partexprs_item = lnext(partexprs_item);
}
- /* Form hash_fn(keyCol) expression */
- fexpr = makeFuncExpr(key->partsupfunc[i].fn_oid,
- get_func_rettype(key->partsupfunc[i].fn_oid),
- list_make2(keyCol, hashSeedConst),
- InvalidOid,
- InvalidOid,
- COERCE_EXPLICIT_CALL);
-
- args = lappend(args, fexpr);
+ args = lappend(args, keyCol);
}
fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
@@ -3405,20 +3393,70 @@ compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
Datum
satisfies_hash_partition(PG_FUNCTION_ARGS)
{
- int modulus = PG_GETARG_INT32(0);
- int remainder = PG_GETARG_INT32(1);
- short nkeys = PG_NARGS() - 2;
+ 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;
+
+ 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);
+ memcpy(my_extra->partsupfunc, key->partsupfunc, nkeys * sizeof(FmgrInfo));
+
+ /* TODO: Should we hold lock until commit? */
+ heap_close(parent, AccessShareLock);
+ }
+
+ /* Get TypeCacheEntry for each partition column. */
for (i = 0; i < nkeys; i++)
+ {
+ /* keys start from fourth argument of function. */
+ int argno = i + 3;
- /*
- * Partition key attribute's hash values start from third argument of
- * function.
- */
- if (!PG_ARGISNULL(i + 2))
- rowHash = hash_combine64(rowHash, PG_GETARG_UINT64(i + 2));
+ 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);
}
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 87acdc0ab3..be9eede93f 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5523,7 +5523,7 @@ DATA(insert OID = 3354 ( pg_ls_waldir PGNSP PGUID 12 10 20 0 0 f f f f t t
DESCR("list of files in the WAL directory");
/* hash partitioning constraint function */
-DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 20 0 f f f f f f i s 3 0 16 "23 23 1016" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 4 0 16 "26 23 23 2276" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
DESCR("hash partition CHECK constraint");
/*
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 5019d1caaf..87aac1ae98 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -749,14 +749,6 @@ Check constraints:
"check_a" CHECK (length(a) > 0)
Partitions: part_c_1_10 FOR VALUES FROM (1) TO (10)
-\d+ hpart_1
- Table "public.hpart_1"
- Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
---------+---------+-----------+----------+---------+---------+--------------+-------------
- a | integer | | | | plain | |
-Partition of: hash_parted FOR VALUES WITH (modulus 10, remainder 0)
-Partition constraint: satisfies_hash_partition(10, 0, hashint4extended(a, '8816678312871386365'::bigint))
-
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
Table "public.part_c_1_10"
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index b08b33d515..901d66c63b 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -666,7 +666,6 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- Both partition bound and partition key in describe output
\d+ part_c
-\d+ hpart_1
-- a level-2 partition's constraint will include the parent's expressions
\d+ part_c_1_10
--
2.14.1
0001-Add-PG_GETARG_UINT64-macro.patchapplication/octet-stream; name=0001-Add-PG_GETARG_UINT64-macro.patchDownload
From a55295a79cabd9db54ccc63d6d6cd9cc90f9e5ca Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 30 Oct 2017 15:34:10 +0530
Subject: [PATCH 1/5] Add PG_GETARG_UINT64 macro
---
src/include/fmgr.h | 1 +
1 file changed, 1 insertion(+)
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index a68ec91c68..96f20d3168 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -245,6 +245,7 @@ extern struct varlena *pg_detoast_datum_packed(struct varlena *datum);
#define PG_GETARG_FLOAT4(n) DatumGetFloat4(PG_GETARG_DATUM(n))
#define PG_GETARG_FLOAT8(n) DatumGetFloat8(PG_GETARG_DATUM(n))
#define PG_GETARG_INT64(n) DatumGetInt64(PG_GETARG_DATUM(n))
+#define PG_GETARG_UINT64(n) DatumGetUInt64(PG_GETARG_DATUM(n))
/* use this if you want the raw, possibly-toasted input datum: */
#define PG_GETARG_RAW_VARLENA_P(n) ((struct varlena *) PG_GETARG_POINTER(n))
/* use this if you want the input datum de-toasted: */
--
2.14.1
0002-partition_bounds_copy-code-refactoring-v1.patchapplication/octet-stream; name=0002-partition_bounds_copy-code-refactoring-v1.patchDownload
From cf0a3cc0d4cd941b82d742d471cb1e0e1ac1046f Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 10 Oct 2017 14:36:24 +0530
Subject: [PATCH 2/5] partition_bounds_copy code refactoring v1
---
src/backend/catalog/partition.c | 35 +++++++++++++++++++++++++++++++++--
1 file changed, 33 insertions(+), 2 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 5daa8a1c19..e900899cc6 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -149,6 +149,7 @@ 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);
/*
* RelationBuildPartitionDesc
@@ -721,8 +722,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);
@@ -2894,3 +2894,34 @@ 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_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;
+}
--
2.14.1
On Wed, Nov 1, 2017 at 3:46 PM, amul sul <sulamul@gmail.com> wrote:
Although partition constraints become more simple, there isn't any performance
gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
copied extended hash function info from the partition key, what if parent is
changed while we are using it? Do we need to keep lock on parent until commit in
satisfies_hash_partition?
I don't think it should be possible for the parent to be changed. I
mean, the partition key is altogether immutable -- it can't be changed
after creation time. The partition bounds can be changed for
individual partitions but that would require a lock on the partition.
Can you give an example of the kind of scenario about which you are concerned?
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Nov 2, 2017 at 1:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Nov 1, 2017 at 3:46 PM, amul sul <sulamul@gmail.com> wrote:
Although partition constraints become more simple, there isn't any performance
gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
copied extended hash function info from the partition key, what if parent is
changed while we are using it? Do we need to keep lock on parent until commit in
satisfies_hash_partition?I don't think it should be possible for the parent to be changed. I
mean, the partition key is altogether immutable -- it can't be changed
after creation time. The partition bounds can be changed for
individual partitions but that would require a lock on the partition.Can you give an example of the kind of scenario about which you are concerned?
Yes, you are correct, column involved in the partitioning are immutable.
I was just worried about any change in the partition key column that
might change selected hash function.
Regards,
Amul
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Nov 2, 2017 at 1:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Nov 1, 2017 at 3:46 PM, amul sul <sulamul@gmail.com> wrote:
Although partition constraints become more simple, there isn't any performance
gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
copied extended hash function info from the partition key, what if parent is
changed while we are using it? Do we need to keep lock on parent until commit in
satisfies_hash_partition?I don't think it should be possible for the parent to be changed. I
mean, the partition key is altogether immutable -- it can't be changed
after creation time. The partition bounds can be changed for
individual partitions but that would require a lock on the partition.Can you give an example of the kind of scenario about which you are concerned?
Right now partition keys are immutable but we don't have much code
written with that assumption. All the code usually keeps a lock on the
parent till the time they use the information in the partition key. In
a distant future, which may not exist, we may support ALTER TABLE ...
PARTITION BY to change partition keys (albeit at huge cost of data
movement). If we do that, we will have to remember this one-off
instance of code which assumes that the partition keys are immutable.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Nov 2, 2017 at 1:45 PM, amul sul <sulamul@gmail.com> wrote:
Yes, you are correct, column involved in the partitioning are immutable.
I was just worried about any change in the partition key column that
might change selected hash function.
Any such change, even if it were allowed, would have to take
AccessExclusiveLock on the child.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Nov 2, 2017 at 1:52 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Right now partition keys are immutable but we don't have much code
written with that assumption. All the code usually keeps a lock on the
parent till the time they use the information in the partition key. In
a distant future, which may not exist, we may support ALTER TABLE ...
PARTITION BY to change partition keys (albeit at huge cost of data
movement). If we do that, we will have to remember this one-off
instance of code which assumes that the partition keys are immutable.
I am pretty sure this is by no means the only piece of code which assumes that.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Nov 1, 2017 at 6:16 AM, amul sul <sulamul@gmail.com> wrote:
Fixed in the 0003 patch.
I have committed this patch set with the attached adjustments.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
hash-adjustments.patchapplication/octet-stream; name=hash-adjustments.patchDownload
commit aca5ff23bb3352f5441e76597e637033a0f0a53d
Author: Robert Haas <rhaas@postgresql.org>
Date: Thu Nov 9 16:32:12 2017 -0500
adjustments
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index b0f1038c20..daba66c187 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2881,11 +2881,10 @@ VALUES ('Albany', NULL, NULL, 'NY');
<listitem>
<para>
- The table is partitioned by specifying a modulus and a remainder for each
- partition. Each partition will hold the rows for which the hash value of
- the partition key divided by the specified modulus will produce the specified
- remainder. Refer to <xref linkend="sql-createtable-partition">
- for additional clarification on modulus and remainder.
+ The table is partitioned by specifying a modulus and a remainder for
+ each partition. Each partition will hold the rows for which the hash
+ value of the partition key divided by the specified modulus will
+ produce the specified remainder.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index b476afd3f7..bbb3a51def 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -257,8 +257,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
Creates the table as a <firstterm>partition</firstterm> of the specified
parent table. The table can be created either as a partition for specific
values using <literal>FOR VALUES</literal> or as a default partition
- using <literal>DEFAULT</literal>. Hash partitioned tables do not support
- a default partition.
+ using <literal>DEFAULT</literal>. This option is not available for
+ hash-partitioned tables.
</para>
<para>
@@ -266,9 +266,9 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
must correspond to the partitioning method and partition key of the
parent table, and must not overlap with any existing partition of that
parent. The form with <literal>IN</literal> is used for list partitioning,
- the form with <literal>FROM</literal> and <literal>TO</literal> is used for
- range partitioning, and the form with <literal>WITH</literal> is used for
- hash partitioning.
+ the form with <literal>FROM</literal> and <literal>TO</literal> is used
+ for range partitioning, and the form with <literal>WITH</literal> is used
+ for hash partitioning.
</para>
<para>
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b8e800db7e..cff59ed055 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -172,8 +172,8 @@ 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 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 */
@@ -547,7 +547,7 @@ RelationBuildPartitionDesc(Relation rel)
case PARTITION_STRATEGY_HASH:
{
/* Modulus are stored in ascending order */
- int greatest_modulus = hbounds[ndatums - 1]->modulus;
+ int greatest_modulus = hbounds[ndatums - 1]->modulus;
boundinfo->indexes = (int *) palloc(greatest_modulus *
sizeof(int));
@@ -755,11 +755,10 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
int greatest_modulus;
/*
- * If two hash partitioned tables have different greatest moduli or
- * same moduli with different number of partitions, 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 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;
@@ -853,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));
@@ -875,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);
@@ -890,26 +889,26 @@ partition_bounds_copy(PartitionBoundInfo src,
for (i = 0; i < ndatums; i++)
{
- int j;
+ int 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;
+ 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;
+ bool byval;
+ int typlen;
if (hash_part)
{
typlen = sizeof(int32); /* Always int4 */
- byval = true; /* int4 is pass-by-value */
+ byval = true; /* int4 is pass-by-value */
}
else
{
@@ -1803,8 +1802,7 @@ make_partition_op_expr(PartitionKey key, int keynum,
* 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
- * hash values computed for each column of the partition key using the extended
- * hash function from the appropriate opclass.
+ * values to be hashed.
*/
static List *
get_qual_for_hash(Relation parent, PartitionBoundSpec *spec)
@@ -2754,9 +2752,9 @@ get_partition_for_tuple(PartitionDispatch *pd,
case PARTITION_STRATEGY_HASH:
{
PartitionBoundInfo boundinfo = partdesc->boundinfo;
- int greatest_modulus = get_greatest_modulus(boundinfo);
- uint64 rowHash = compute_hash_value(key, values,
- isnull);
+ int greatest_modulus = get_greatest_modulus(boundinfo);
+ uint64 rowHash = compute_hash_value(key, values,
+ isnull);
cur_index = boundinfo->indexes[rowHash % greatest_modulus];
}
@@ -3294,13 +3292,14 @@ get_proposed_default_constraint(List *new_part_constraints)
static int
get_partition_bound_num_indexes(PartitionBoundInfo bound)
{
- int num_indexes;
+ 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.
@@ -3359,7 +3358,7 @@ compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
{
if (!isnull[i])
{
- Datum hash;
+ Datum hash;
Assert(OidIsValid(key->partsupfunc[i].fn_oid));
@@ -3397,7 +3396,7 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
{
Oid relid;
int16 nkeys;
- FmgrInfo partsupfunc[PARTITION_MAX_KEYS];
+ FmgrInfo partsupfunc[PARTITION_MAX_KEYS];
} ColumnsHashData;
Oid parentId = PG_GETARG_OID(0);
int modulus = PG_GETARG_INT32(1);
@@ -3417,6 +3416,7 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
{
Relation parent;
PartitionKey key;
+ int j;
fcinfo->flinfo->fn_extra =
MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
@@ -3431,17 +3431,19 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
key = RelationGetPartitionKey(parent);
Assert(key->partnatts == nkeys);
- memcpy(my_extra->partsupfunc, key->partsupfunc, nkeys * sizeof(FmgrInfo));
+ for (j = 0; j < nkeys; ++j)
+ fmgr_info_copy(&my_extra->partsupfunc[j],
+ key->partsupfunc,
+ fcinfo->flinfo->fn_mcxt);
- /* TODO: Should we hold lock until commit? */
- heap_close(parent, AccessShareLock);
+ /* Hold lock until commit */
+ heap_close(parent, NoLock);
}
- /* Get TypeCacheEntry for each partition column. */
for (i = 0; i < nkeys; i++)
{
/* keys start from fourth argument of function. */
- int argno = i + 3;
+ int argno = i + 3;
if (!PG_ARGISNULL(argno))
{
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 831eb05644..453f25964a 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -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;
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 542b298fd0..8461da490a 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3343,7 +3343,7 @@ transformPartitionBound(ParseState *pstate, Relation parent,
if (spec->remainder >= spec->modulus)
ereport(ERROR,
(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
- errmsg("modulus for hash partition must be greater than remainder")));
+ errmsg("remainder for hash partition must be less than modulus")));
}
else if (strategy == PARTITION_STRATEGY_LIST)
{
@@ -3510,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)
diff --git a/src/include/catalog/catversion.h b/src/include/catalog/catversion.h
index 39c70b415a..bd4014a69d 100644
--- a/src/include/catalog/catversion.h
+++ b/src/include/catalog/catversion.h
@@ -53,6 +53,6 @@
*/
/* yyyymmddN */
-#define CATALOG_VERSION_NO 201711091
+#define CATALOG_VERSION_NO 201711092
#endif
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 96f20d3168..a68ec91c68 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -245,7 +245,6 @@ extern struct varlena *pg_detoast_datum_packed(struct varlena *datum);
#define PG_GETARG_FLOAT4(n) DatumGetFloat4(PG_GETARG_DATUM(n))
#define PG_GETARG_FLOAT8(n) DatumGetFloat8(PG_GETARG_DATUM(n))
#define PG_GETARG_INT64(n) DatumGetInt64(PG_GETARG_DATUM(n))
-#define PG_GETARG_UINT64(n) DatumGetUInt64(PG_GETARG_DATUM(n))
/* use this if you want the raw, possibly-toasted input datum: */
#define PG_GETARG_RAW_VARLENA_P(n) ((struct varlena *) PG_GETARG_POINTER(n))
/* use this if you want the input datum de-toasted: */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index b4af2227e3..11f0baa11b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3598,7 +3598,7 @@ CREATE TABLE quuux2 PARTITION OF quuux FOR VALUES IN (2);
INFO: updated partition constraint for default partition "quuux_default1" is implied by existing constraints
DROP TABLE quuux;
-- check validation when attaching hash partitions
--- The default hash functions as they exist today aren't portable, they can
+-- The default hash functions as they exist today aren't portable; they can
-- return different results on different machines. Depending upon how the
-- values are hashed, the row may map to different partitions, which result in
-- regression failure. To avoid this, let's create a non-default hash function
@@ -3646,7 +3646,7 @@ CREATE TABLE fail_part(LIKE hash_parted);
ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 0, REMAINDER 1);
ERROR: modulus for hash partition must be a positive integer
ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 8, REMAINDER 8);
-ERROR: modulus for hash partition must be greater than remainder
+ERROR: remainder for hash partition must be less than modulus
ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (MODULUS 3, REMAINDER 2);
ERROR: every hash partition modulus must be a factor of the next larger modulus
DROP TABLE fail_part;
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 87aac1ae98..335cd37e18 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,6 +340,11 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
ERROR: cannot use constant expression as partition key
DROP FUNCTION const_func();
+-- only accept valid partitioning strategy
+CREATE TABLE partitioned (
+ a int
+) PARTITION BY MAGIC (a);
+ERROR: unrecognized partitioning strategy "magic"
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
@@ -677,7 +682,7 @@ CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 0, REM
ERROR: modulus for hash partition must be a positive integer
-- remainder must be greater than or equal to zero and less than modulus
CREATE TABLE fail_part PARTITION OF hash_parted2 FOR VALUES WITH (MODULUS 8, REMAINDER 8);
-ERROR: modulus for hash partition must be greater than remainder
+ERROR: remainder for hash partition must be less than modulus
-- check schema propagation from parent
CREATE TABLE parted (
a text,
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index da1c1441bd..02a33ca7c4 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2335,7 +2335,7 @@ DROP TABLE quuux;
-- check validation when attaching hash partitions
--- The default hash functions as they exist today aren't portable, they can
+-- The default hash functions as they exist today aren't portable; they can
-- return different results on different machines. Depending upon how the
-- values are hashed, the row may map to different partitions, which result in
-- regression failure. To avoid this, let's create a non-default hash function
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index 901d66c63b..b77b476436 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,6 +350,11 @@ CREATE TABLE partitioned (
) PARTITION BY RANGE (const_func());
DROP FUNCTION const_func();
+-- only accept valid partitioning strategy
+CREATE TABLE partitioned (
+ a int
+) PARTITION BY MAGIC (a);
+
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
On Fri, Nov 10, 2017 at 4:41 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Nov 1, 2017 at 6:16 AM, amul sul <sulamul@gmail.com> wrote:
Fixed in the 0003 patch.
I have committed this patch set with the attached adjustments.
Thanks a lot for your support & a ton of thanks to all reviewer.
Regards,
Amul
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers