[POC] hash partitioning
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.
* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.
The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.
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.
When trying create partitions more than the number specified
by PARTITIONS, it gets an error.
postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for h
An inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.
postgres=# insert into h (select generate_series(0,20));
INSERT 0 21
postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)
* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.
In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.
Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".
postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)
However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().
postgres=# explain select * from h where abs(hashint4(i))%3 = abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)
Best regards,
Yugo Nagata
--
Yugo Nagata <nagata@sraoss.co.jp>
Attachments:
hash_partition.patchtext/x-diff; name=hash_partition.patchDownload
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 41c0056..3820920 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -3074,7 +3074,7 @@ StorePartitionKey(Relation rel,
AttrNumber *partattrs,
List *partexprs,
Oid *partopclass,
- Oid *partcollation)
+ Oid *partcollation, int16 partnparts, Oid hashfunc)
{
int i;
int2vector *partattrs_vec;
@@ -3121,6 +3121,8 @@ StorePartitionKey(Relation rel,
values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel));
values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy);
values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts);
+ values[Anum_pg_partitioned_table_partnparts - 1] = Int16GetDatum(partnparts);
+ values[Anum_pg_partitioned_table_parthashfunc - 1] = ObjectIdGetDatum(hashfunc);
values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec);
values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec);
values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec);
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 4bcef58..24e69c6 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -36,6 +36,8 @@
#include "optimizer/clauses.h"
#include "optimizer/planmain.h"
#include "optimizer/var.h"
+#include "parser/parse_func.h"
+#include "parser/parse_oper.h"
#include "rewrite/rewriteManip.h"
#include "storage/lmgr.h"
#include "utils/array.h"
@@ -120,6 +122,7 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
+static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec);
static Oid get_partition_operator(PartitionKey key, int col,
StrategyNumber strategy, bool *need_relabel);
static List *generate_partition_qual(Relation rel);
@@ -236,7 +239,8 @@ 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_LIST ||
+ key->strategy == PARTITION_STRATEGY_HASH)
{
List *non_null_values = NIL;
@@ -251,7 +255,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
+ if (spec->strategy != key->strategy)
elog(ERROR, "invalid strategy in partition bound spec");
foreach(c, spec->listdatums)
@@ -464,6 +468,7 @@ RelationBuildPartitionDesc(Relation rel)
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
{
boundinfo->has_null = found_null;
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
@@ -829,6 +834,18 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
break;
}
+ case PARTITION_STRATEGY_HASH:
+ {
+ Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+
+ if (partdesc->nparts + 1 > key->partnparts)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+ errmsg("cannot create hash partition more than %d for %s",
+ key->partnparts, RelationGetRelationName(parent))));
+ break;
+ }
+
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
@@ -916,6 +933,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
my_qual = get_qual_for_range(key, spec);
break;
+ case PARTITION_STRATEGY_HASH:
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+ my_qual = get_qual_for_hash(key, spec);
+ break;
+
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
@@ -1146,6 +1168,84 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
return pd;
}
+/*
+ * convert_expr_for_hash
+ *
+ * Converts a expr for a hash partition's constraint.
+ * expr is converted into 'abs(hashfunc(expr)) % npart".
+ *
+ * npart: number of partitions
+ * hashfunc: OID of hash function
+ */
+Expr *
+convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc)
+{
+ FuncExpr *func,
+ *abs;
+ Expr *modexpr;
+ Oid modoid;
+ Oid int4oid[1] = {INT4OID};
+
+ ParseState *pstate = make_parsestate(NULL);
+ Value *val_npart = makeInteger(npart);
+ Node *const_npart = (Node *) make_const(pstate, val_npart, -1);
+
+ /* hash function */
+ func = makeFuncExpr(hashfunc,
+ INT4OID,
+ list_make1(expr),
+ 0,
+ 0,
+ COERCE_EXPLICIT_CALL);
+
+ /* Abs */
+ abs = makeFuncExpr(LookupFuncName(list_make1(makeString("abs")), 1, int4oid, false),
+ INT4OID,
+ list_make1(func),
+ 0,
+ 0,
+ COERCE_EXPLICIT_CALL);
+
+ /* modulo by npart */
+ modoid = LookupOperName(pstate, list_make1(makeString("%")), INT4OID, INT4OID, false, -1);
+ modexpr = make_opclause(modoid, INT4OID, false, (Expr*)abs, (Expr*)const_npart, 0, 0);
+
+ return modexpr;
+}
+
+
+/*
+ * get_next_hash_partition_index
+ *
+ * Returns the minimal index which is not used for hash partition.
+ */
+int
+get_next_hash_partition_index(Relation parent)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+
+ int i;
+ bool *used = palloc0(sizeof(int) * key->partnparts);
+
+ /* mark used for existing partition indexs */
+ for (i = 0; i < partdesc->boundinfo->ndatums; i++)
+ {
+ Datum* datum = partdesc->boundinfo->datums[i];
+ int idx = DatumGetInt16(datum[0]);
+
+ if (!used[idx])
+ used[idx] = true;
+ }
+
+ /* find the minimal unused index */
+ for (i = 0; i < key->partnparts; i++)
+ if (!used[i])
+ break;
+
+ return i;
+}
+
/* Module-local functions */
/*
@@ -1467,6 +1567,43 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
}
/*
+ * get_qual_for_hash
+ *
+ * Returns a list of expressions to use as a hash partition's constraint.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ List *result;
+ Expr *keyCol;
+ Expr *expr;
+ Expr *opexpr;
+ Oid operoid;
+ ParseState *pstate = make_parsestate(NULL);
+
+ /* Left operand */
+ if (key->partattrs[0] != 0)
+ keyCol = (Expr *) makeVar(1,
+ key->partattrs[0],
+ key->parttypid[0],
+ key->parttypmod[0],
+ key->parttypcoll[0],
+ 0);
+ else
+ keyCol = (Expr *) copyObject(linitial(key->partexprs));
+
+ expr = convert_expr_for_hash(keyCol, key->partnparts, key->parthashfunc);
+
+ /* equals the listdaums value */
+ operoid = LookupOperName(pstate, list_make1(makeString("=")), INT4OID, INT4OID, false, -1);
+ opexpr = make_opclause(operoid, BOOLOID, false, expr, linitial(spec->listdatums), 0, 0);
+
+ result = list_make1(opexpr);
+
+ return result;
+}
+
+/*
* get_partition_operator
*
* Return oid of the operator of given strategy for a given partition key
@@ -1730,6 +1867,11 @@ get_partition_for_tuple(PartitionDispatch *pd,
(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("range partition key of row contains null")));
}
+ else if (key->strategy == PARTITION_STRATEGY_HASH)
+ {
+ values[0] = OidFunctionCall1(key->parthashfunc, values[0]);
+ values[0] = Int16GetDatum(Abs(DatumGetInt16(values[0])) % key->partnparts);
+ }
if (partdesc->boundinfo->has_null && isnull[0])
/* Tuple maps to the null-accepting list partition */
@@ -1744,6 +1886,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
if (cur_offset >= 0 && equal)
cur_index = partdesc->boundinfo->indexes[cur_offset];
else
@@ -1968,6 +2111,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
bound_datums[0],
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3cea220..5a28cc0 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -41,6 +41,7 @@
#include "catalog/pg_inherits_fn.h"
#include "catalog/pg_namespace.h"
#include "catalog/pg_opclass.h"
+#include "catalog/pg_proc.h"
#include "catalog/pg_tablespace.h"
#include "catalog/pg_trigger.h"
#include "catalog/pg_type.h"
@@ -77,6 +78,7 @@
#include "parser/parse_oper.h"
#include "parser/parse_relation.h"
#include "parser/parse_type.h"
+#include "parser/parse_func.h"
#include "parser/parse_utilcmd.h"
#include "parser/parser.h"
#include "pgstat.h"
@@ -450,7 +452,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid,
Oid oldrelid, void *arg);
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,
+static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes,
List **partexprs, Oid *partopclass, Oid *partcollation);
static void CreateInheritance(Relation child_rel, Relation parent_rel);
static void RemoveInheritance(Relation child_rel, Relation parent_rel);
@@ -799,8 +801,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
AttrNumber partattrs[PARTITION_MAX_KEYS];
Oid partopclass[PARTITION_MAX_KEYS];
Oid partcollation[PARTITION_MAX_KEYS];
+ Oid partatttypes[PARTITION_MAX_KEYS];
List *partexprs = NIL;
List *cmds = NIL;
+ Oid hashfuncOid = InvalidOid;
/*
* We need to transform the raw parsetrees corresponding to partition
@@ -811,15 +815,40 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
stmt->partspec = transformPartitionSpec(rel, stmt->partspec,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
- partattrs, &partexprs, partopclass,
+ partattrs, partatttypes, &partexprs, partopclass,
partcollation);
partnatts = list_length(stmt->partspec->partParams);
+
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ Oid funcrettype;
+
+ if (partnatts != 1)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partition key must be 1 for hash partition")));
+
+ hashfuncOid = LookupFuncName(stmt->partspec->hashfunc, 1, partatttypes, false);
+ funcrettype = get_func_rettype(hashfuncOid);
+ if (funcrettype != INT4OID)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function for partitioning must return integer")));
+
+ if (func_volatile(hashfuncOid) != PROVOLATILE_IMMUTABLE)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function for partitioning must be marked IMMUTABLE")));
+
+ }
+
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
- partopclass, partcollation);
+ partopclass, partcollation, stmt->partspec->partnparts, hashfuncOid);
- /* Force key columns to be NOT NULL when using range partitioning */
- if (strategy == PARTITION_STRATEGY_RANGE)
+ /* Force key columns to be NOT NULL when using range or hash partitioning */
+ if (strategy == PARTITION_STRATEGY_RANGE ||
+ strategy == PARTITION_STRATEGY_HASH)
{
for (i = 0; i < partnatts; i++)
{
@@ -12783,18 +12812,51 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
newspec->strategy = partspec->strategy;
newspec->location = partspec->location;
newspec->partParams = NIL;
+ newspec->partnparts = partspec->partnparts;
+ newspec->hashfunc = partspec->hashfunc;
/* Parse partitioning strategy name */
if (!pg_strcasecmp(partspec->strategy, "list"))
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("unrecognized partitioning strategy \"%s\"",
partspec->strategy)));
+ if (*strategy == PARTITION_STRATEGY_HASH)
+ {
+ if (partspec->partnparts < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partitions must be specified for hash partition")));
+ else if (partspec->partnparts == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partitions must be greater than 0")));
+
+ if (list_length(partspec->hashfunc) == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function must be specified for hash partition")));
+ }
+ else
+ {
+ if (partspec->partnparts >= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partitions can be specified only for hash partition")));
+
+ if (list_length(partspec->hashfunc) > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function can be specified only for hash partition")));
+ }
+
/*
* Create a dummy ParseState and insert the target relation as its sole
* rangetable entry. We need a ParseState for transformExpr.
@@ -12843,7 +12905,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
* Compute per-partition-column information from a list of PartitionElem's
*/
static void
-ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
+ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes,
List **partexprs, Oid *partopclass, Oid *partcollation)
{
int attn;
@@ -13010,6 +13072,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
"btree",
BTREE_AM_OID);
+ partatttypes[attn] = atttype;
attn++;
}
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 05d8538..f4febc9 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4232,6 +4232,8 @@ _copyPartitionSpec(const PartitionSpec *from)
COPY_STRING_FIELD(strategy);
COPY_NODE_FIELD(partParams);
+ COPY_SCALAR_FIELD(partnparts);
+ COPY_NODE_FIELD(hashfunc);
COPY_LOCATION_FIELD(location);
return newnode;
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index d595cd7..d589eac 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2725,6 +2725,8 @@ _equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b)
{
COMPARE_STRING_FIELD(strategy);
COMPARE_NODE_FIELD(partParams);
+ COMPARE_SCALAR_FIELD(partnparts);
+ COMPARE_NODE_FIELD(hashfunc);
COMPARE_LOCATION_FIELD(location);
return true;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..d6db80e 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3318,6 +3318,8 @@ _outPartitionSpec(StringInfo str, const PartitionSpec *node)
WRITE_STRING_FIELD(strategy);
WRITE_NODE_FIELD(partParams);
+ WRITE_INT_FIELD(partnparts);
+ WRITE_NODE_FIELD(hashfunc);
WRITE_LOCATION_FIELD(location);
}
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index e833b2e..b67140d 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -574,6 +574,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
%type <list> range_datum_list
+%type <ival> hash_partitions
+%type <list> hash_function
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -627,7 +629,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING
- HANDLER HAVING HEADER_P HOLD HOUR_P
+ HANDLER HASH HAVING HEADER_P HOLD HOUR_P
IDENTITY_P IF_P ILIKE IMMEDIATE IMMUTABLE IMPLICIT_P IMPORT_P IN_P
INCLUDING INCREMENT INDEX INDEXES INHERIT INHERITS INITIALLY INLINE_P
@@ -651,7 +653,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
OBJECT_P OF OFF OFFSET OIDS OLD ON ONLY OPERATOR OPTION OPTIONS OR
ORDER ORDINALITY OUT_P OUTER_P OVER OVERLAPS OVERLAY OWNED OWNER
- PARALLEL PARSER PARTIAL PARTITION PASSING PASSWORD PLACING PLANS POLICY
+ PARALLEL PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PLACING PLANS POLICY
POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY
PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROGRAM PUBLICATION
@@ -2587,6 +2589,16 @@ ForValues:
$$ = (Node *) n;
}
+
+ /* a HASH partition */
+ | /*EMPTY*/
+ {
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+
+ $$ = (Node *) n;
+ }
;
partbound_datum:
@@ -3666,7 +3678,7 @@ OptPartitionSpec: PartitionSpec { $$ = $1; }
| /*EMPTY*/ { $$ = NULL; }
;
-PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
+PartitionSpec: PARTITION BY part_strategy '(' part_params ')' hash_partitions hash_function
{
PartitionSpec *n = makeNode(PartitionSpec);
@@ -3674,10 +3686,21 @@ PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
n->partParams = $5;
n->location = @1;
+ n->partnparts = $7;
+ n->hashfunc = $8;
+
$$ = n;
}
;
+hash_partitions: PARTITIONS Iconst { $$ = $2; }
+ | /*EMPTY*/ { $$ = -1; }
+ ;
+
+hash_function: USING handler_name { $$ = $2; }
+ | /*EMPTY*/ { $$ = NULL; }
+ ;
+
part_strategy: IDENT { $$ = $1; }
| unreserved_keyword { $$ = pstrdup($1); }
;
@@ -14377,6 +14400,7 @@ unreserved_keyword:
| GLOBAL
| GRANTED
| HANDLER
+ | HASH
| HEADER_P
| HOLD
| HOUR_P
@@ -14448,6 +14472,7 @@ unreserved_keyword:
| PARSER
| PARTIAL
| PARTITION
+ | PARTITIONS
| PASSING
| PASSWORD
| PLANS
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index ff2bab6..8e1be31 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -40,6 +40,7 @@
#include "catalog/pg_opclass.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_type.h"
+#include "catalog/partition.h"
#include "commands/comment.h"
#include "commands/defrem.h"
#include "commands/tablecmds.h"
@@ -3252,6 +3253,24 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
++i;
}
}
+ else if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ Value *conval;
+ Node *value;
+ int index;
+
+ if (spec->strategy != PARTITION_STRATEGY_HASH)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition")));
+
+ index = get_next_hash_partition_index(parent);
+
+ /* store the partition index as a listdatums value */
+ conval = makeInteger(index);
+ value = (Node *) make_const(pstate, conval, -1);
+ result_spec->listdatums = list_make1(value);
+ }
else
elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index b27b77d..fab6eea 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1423,7 +1423,7 @@ pg_get_indexdef_worker(Oid indexrelid, int colno,
*
* 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)
@@ -1513,6 +1513,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags)
case PARTITION_STRATEGY_RANGE:
appendStringInfo(&buf, "RANGE");
break;
+ case PARTITION_STRATEGY_HASH:
+ appendStringInfo(&buf, "HASH");
+ break;
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) form->partstrat);
@@ -8520,6 +8523,9 @@ get_rule_expr(Node *node, deparse_context *context,
appendStringInfoString(buf, ")");
break;
+ case PARTITION_STRATEGY_HASH:
+ break;
+
default:
elog(ERROR, "unrecognized partition strategy: %d",
(int) spec->strategy);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 9001e20..829e4d2 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -855,6 +855,9 @@ RelationBuildPartitionKey(Relation relation)
key->strategy = form->partstrat;
key->partnatts = form->partnatts;
+ key->partnparts = form->partnparts;
+ key->parthashfunc = form->parthashfunc;
+
/*
* We can rely on the first variable-length attribute being mapped to the
* relevant field of the catalog's C struct, because all previous
@@ -999,6 +1002,9 @@ copy_partition_key(PartitionKey fromkey)
newkey->strategy = fromkey->strategy;
newkey->partnatts = n = fromkey->partnatts;
+ newkey->partnparts = fromkey->partnparts;
+ newkey->parthashfunc = fromkey->parthashfunc;
+
newkey->partattrs = (AttrNumber *) palloc(n * sizeof(AttrNumber));
memcpy(newkey->partattrs, fromkey->partattrs, n * sizeof(AttrNumber));
diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h
index 1187797..367e2f8 100644
--- a/src/include/catalog/heap.h
+++ b/src/include/catalog/heap.h
@@ -141,7 +141,7 @@ extern void StorePartitionKey(Relation rel,
AttrNumber *partattrs,
List *partexprs,
Oid *partopclass,
- Oid *partcollation);
+ Oid *partcollation, int16 partnparts, Oid hashfunc);
extern void RemovePartitionKeyByRelId(Oid relid);
extern void StorePartitionBound(Relation rel, Relation parent, Node *bound);
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index b195d1a..80f4b0e 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -89,4 +89,6 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
TupleTableSlot *slot,
EState *estate,
Oid *failed_at);
+extern Expr *convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc);
+extern int get_next_hash_partition_index(Relation parent);
#endif /* PARTITION_H */
diff --git a/src/include/catalog/pg_partitioned_table.h b/src/include/catalog/pg_partitioned_table.h
index bdff36a..69e509c 100644
--- a/src/include/catalog/pg_partitioned_table.h
+++ b/src/include/catalog/pg_partitioned_table.h
@@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
char partstrat; /* partitioning strategy */
int16 partnatts; /* number of partition key columns */
+ int16 partnparts;
+ Oid parthashfunc;
+
/*
* variable-length fields start here, but we allow direct access to
* partattrs via the C struct. That's because the first variable-length
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;
/* ----------------
@@ -62,13 +67,15 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table;
* compiler constants for pg_partitioned_table
* ----------------
*/
-#define Natts_pg_partitioned_table 7
+#define Natts_pg_partitioned_table 9
#define Anum_pg_partitioned_table_partrelid 1
#define Anum_pg_partitioned_table_partstrat 2
#define Anum_pg_partitioned_table_partnatts 3
-#define Anum_pg_partitioned_table_partattrs 4
-#define Anum_pg_partitioned_table_partclass 5
-#define Anum_pg_partitioned_table_partcollation 6
-#define Anum_pg_partitioned_table_partexprs 7
+#define Anum_pg_partitioned_table_partnparts 4
+#define Anum_pg_partitioned_table_parthashfunc 5
+#define Anum_pg_partitioned_table_partattrs 6
+#define Anum_pg_partitioned_table_partclass 7
+#define Anum_pg_partitioned_table_partcollation 8
+#define Anum_pg_partitioned_table_partexprs 9
#endif /* PG_PARTITIONED_TABLE_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 5afc3eb..1c3474f 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -730,11 +730,14 @@ typedef struct PartitionSpec
NodeTag type;
char *strategy; /* partitioning strategy ('list' or 'range') */
List *partParams; /* List of PartitionElems */
+ int partnparts;
+ List *hashfunc;
int location; /* token location, or -1 if unknown */
} PartitionSpec;
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'
/*
* PartitionBoundSpec - a partition bound specification
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 985d650..0597939 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -180,6 +180,7 @@ PG_KEYWORD("greatest", GREATEST, COL_NAME_KEYWORD)
PG_KEYWORD("group", GROUP_P, RESERVED_KEYWORD)
PG_KEYWORD("grouping", GROUPING, COL_NAME_KEYWORD)
PG_KEYWORD("handler", HANDLER, UNRESERVED_KEYWORD)
+PG_KEYWORD("hash", HASH, UNRESERVED_KEYWORD)
PG_KEYWORD("having", HAVING, RESERVED_KEYWORD)
PG_KEYWORD("header", HEADER_P, UNRESERVED_KEYWORD)
PG_KEYWORD("hold", HOLD, UNRESERVED_KEYWORD)
@@ -291,6 +292,7 @@ PG_KEYWORD("parallel", PARALLEL, UNRESERVED_KEYWORD)
PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD)
PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD)
PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD)
+PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD)
PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD)
PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD)
PG_KEYWORD("placing", PLACING, RESERVED_KEYWORD)
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index a617a7c..660adfb 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -62,6 +62,9 @@ typedef struct PartitionKeyData
Oid *partopcintype; /* OIDs of opclass declared input data types */
FmgrInfo *partsupfunc; /* lookup info for support funcs */
+ int16 partnparts; /* number of hash partitions */
+ Oid parthashfunc; /* OID of hash function */
+
/* Partitioning collation per attribute */
Oid *partcollation;
Hi, Yugo.
Looks like a great feature! I'm going to take a closer look on your code
and write a feedback shortly. For now I can only tell that you forgot
to include some documentation in the patch.
I've added a corresponding entry to current commitfest [1]https://commitfest.postgresql.org/13/1059/. Hope you
don't mind. If it's not too much trouble could you please register on a
commitfest site and add yourself to this entry as an author? I'm pretty
sure someone is using this information for writing release notes or
something like this.
[1]: https://commitfest.postgresql.org/13/1059/
On Tue, Feb 28, 2017 at 11:33:13PM +0900, Yugo Nagata wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().postgres=# explain select * from h where abs(hashint4(i))%3 = abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)Best regards,
Yugo Nagata--
Yugo Nagata <nagata@sraoss.co.jp>
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c index 41c0056..3820920 100644 --- a/src/backend/catalog/heap.c +++ b/src/backend/catalog/heap.c @@ -3074,7 +3074,7 @@ StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation) + Oid *partcollation, int16 partnparts, Oid hashfunc) { int i; int2vector *partattrs_vec; @@ -3121,6 +3121,8 @@ StorePartitionKey(Relation rel, values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel)); values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy); values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts); + values[Anum_pg_partitioned_table_partnparts - 1] = Int16GetDatum(partnparts); + values[Anum_pg_partitioned_table_parthashfunc - 1] = ObjectIdGetDatum(hashfunc); values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec); values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec); values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec); diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c index 4bcef58..24e69c6 100644 --- a/src/backend/catalog/partition.c +++ b/src/backend/catalog/partition.c @@ -36,6 +36,8 @@ #include "optimizer/clauses.h" #include "optimizer/planmain.h" #include "optimizer/var.h" +#include "parser/parse_func.h" +#include "parser/parse_oper.h" #include "rewrite/rewriteManip.h" #include "storage/lmgr.h" #include "utils/array.h" @@ -120,6 +122,7 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec); static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec); +static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec); static Oid get_partition_operator(PartitionKey key, int col, StrategyNumber strategy, bool *need_relabel); static List *generate_partition_qual(Relation rel); @@ -236,7 +239,8 @@ 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_LIST || + key->strategy == PARTITION_STRATEGY_HASH) { List *non_null_values = NIL;@@ -251,7 +255,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);- if (spec->strategy != PARTITION_STRATEGY_LIST) + if (spec->strategy != key->strategy) elog(ERROR, "invalid strategy in partition bound spec");foreach(c, spec->listdatums)
@@ -464,6 +468,7 @@ RelationBuildPartitionDesc(Relation rel)
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
{
boundinfo->has_null = found_null;
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
@@ -829,6 +834,18 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
break;
}+ case PARTITION_STRATEGY_HASH: + { + Assert(spec->strategy == PARTITION_STRATEGY_HASH); + + if (partdesc->nparts + 1 > key->partnparts) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("cannot create hash partition more than %d for %s", + key->partnparts, RelationGetRelationName(parent)))); + break; + } + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -916,6 +933,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound) my_qual = get_qual_for_range(key, spec); break;+ case PARTITION_STRATEGY_HASH: + Assert(spec->strategy == PARTITION_STRATEGY_LIST); + my_qual = get_qual_for_hash(key, spec); + break; + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -1146,6 +1168,84 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode, return pd; }+/* + * convert_expr_for_hash + * + * Converts a expr for a hash partition's constraint. + * expr is converted into 'abs(hashfunc(expr)) % npart". + * + * npart: number of partitions + * hashfunc: OID of hash function + */ +Expr * +convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc) +{ + FuncExpr *func, + *abs; + Expr *modexpr; + Oid modoid; + Oid int4oid[1] = {INT4OID}; + + ParseState *pstate = make_parsestate(NULL); + Value *val_npart = makeInteger(npart); + Node *const_npart = (Node *) make_const(pstate, val_npart, -1); + + /* hash function */ + func = makeFuncExpr(hashfunc, + INT4OID, + list_make1(expr), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* Abs */ + abs = makeFuncExpr(LookupFuncName(list_make1(makeString("abs")), 1, int4oid, false), + INT4OID, + list_make1(func), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* modulo by npart */ + modoid = LookupOperName(pstate, list_make1(makeString("%")), INT4OID, INT4OID, false, -1); + modexpr = make_opclause(modoid, INT4OID, false, (Expr*)abs, (Expr*)const_npart, 0, 0); + + return modexpr; +} + + +/* + * get_next_hash_partition_index + * + * Returns the minimal index which is not used for hash partition. + */ +int +get_next_hash_partition_index(Relation parent) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionDesc partdesc = RelationGetPartitionDesc(parent); + + int i; + bool *used = palloc0(sizeof(int) * key->partnparts); + + /* mark used for existing partition indexs */ + for (i = 0; i < partdesc->boundinfo->ndatums; i++) + { + Datum* datum = partdesc->boundinfo->datums[i]; + int idx = DatumGetInt16(datum[0]); + + if (!used[idx]) + used[idx] = true; + } + + /* find the minimal unused index */ + for (i = 0; i < key->partnparts; i++) + if (!used[i]) + break; + + return i; +} + /* Module-local functions *//*
@@ -1467,6 +1567,43 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
}/* + * get_qual_for_hash + * + * Returns a list of expressions to use as a hash partition's constraint. + */ +static List * +get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec) +{ + List *result; + Expr *keyCol; + Expr *expr; + Expr *opexpr; + Oid operoid; + ParseState *pstate = make_parsestate(NULL); + + /* Left operand */ + if (key->partattrs[0] != 0) + keyCol = (Expr *) makeVar(1, + key->partattrs[0], + key->parttypid[0], + key->parttypmod[0], + key->parttypcoll[0], + 0); + else + keyCol = (Expr *) copyObject(linitial(key->partexprs)); + + expr = convert_expr_for_hash(keyCol, key->partnparts, key->parthashfunc); + + /* equals the listdaums value */ + operoid = LookupOperName(pstate, list_make1(makeString("=")), INT4OID, INT4OID, false, -1); + opexpr = make_opclause(operoid, BOOLOID, false, expr, linitial(spec->listdatums), 0, 0); + + result = list_make1(opexpr); + + return result; +} + +/* * get_partition_operator * * Return oid of the operator of given strategy for a given partition key @@ -1730,6 +1867,11 @@ get_partition_for_tuple(PartitionDispatch *pd, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), errmsg("range partition key of row contains null"))); } + else if (key->strategy == PARTITION_STRATEGY_HASH) + { + values[0] = OidFunctionCall1(key->parthashfunc, values[0]); + values[0] = Int16GetDatum(Abs(DatumGetInt16(values[0])) % key->partnparts); + }if (partdesc->boundinfo->has_null && isnull[0]) /* Tuple maps to the null-accepting list partition */ @@ -1744,6 +1886,7 @@ get_partition_for_tuple(PartitionDispatch *pd, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: if (cur_offset >= 0 && equal) cur_index = partdesc->boundinfo->indexes[cur_offset]; else @@ -1968,6 +2111,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0], key->partcollation[0], bound_datums[0], diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index 3cea220..5a28cc0 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -41,6 +41,7 @@ #include "catalog/pg_inherits_fn.h" #include "catalog/pg_namespace.h" #include "catalog/pg_opclass.h" +#include "catalog/pg_proc.h" #include "catalog/pg_tablespace.h" #include "catalog/pg_trigger.h" #include "catalog/pg_type.h" @@ -77,6 +78,7 @@ #include "parser/parse_oper.h" #include "parser/parse_relation.h" #include "parser/parse_type.h" +#include "parser/parse_func.h" #include "parser/parse_utilcmd.h" #include "parser/parser.h" #include "pgstat.h" @@ -450,7 +452,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid, Oid oldrelid, void *arg); 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, +static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation); static void CreateInheritance(Relation child_rel, Relation parent_rel); static void RemoveInheritance(Relation child_rel, Relation parent_rel); @@ -799,8 +801,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, AttrNumber partattrs[PARTITION_MAX_KEYS]; Oid partopclass[PARTITION_MAX_KEYS]; Oid partcollation[PARTITION_MAX_KEYS]; + Oid partatttypes[PARTITION_MAX_KEYS]; List *partexprs = NIL; List *cmds = NIL; + Oid hashfuncOid = InvalidOid;/* * We need to transform the raw parsetrees corresponding to partition @@ -811,15 +815,40 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, stmt->partspec = transformPartitionSpec(rel, stmt->partspec, &strategy); ComputePartitionAttrs(rel, stmt->partspec->partParams, - partattrs, &partexprs, partopclass, + partattrs, partatttypes, &partexprs, partopclass, partcollation);partnatts = list_length(stmt->partspec->partParams); + + if (strategy == PARTITION_STRATEGY_HASH) + { + Oid funcrettype; + + if (partnatts != 1) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partition key must be 1 for hash partition"))); + + hashfuncOid = LookupFuncName(stmt->partspec->hashfunc, 1, partatttypes, false); + funcrettype = get_func_rettype(hashfuncOid); + if (funcrettype != INT4OID) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must return integer"))); + + if (func_volatile(hashfuncOid) != PROVOLATILE_IMMUTABLE) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must be marked IMMUTABLE"))); + + } + StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs, - partopclass, partcollation); + partopclass, partcollation, stmt->partspec->partnparts, hashfuncOid);- /* Force key columns to be NOT NULL when using range partitioning */ - if (strategy == PARTITION_STRATEGY_RANGE) + /* Force key columns to be NOT NULL when using range or hash partitioning */ + if (strategy == PARTITION_STRATEGY_RANGE || + strategy == PARTITION_STRATEGY_HASH) { for (i = 0; i < partnatts; i++) { @@ -12783,18 +12812,51 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) newspec->strategy = partspec->strategy; newspec->location = partspec->location; newspec->partParams = NIL; + newspec->partnparts = partspec->partnparts; + newspec->hashfunc = partspec->hashfunc;/* Parse partitioning strategy name */ if (!pg_strcasecmp(partspec->strategy, "list")) *strategy = PARTITION_STRATEGY_LIST; else if (!pg_strcasecmp(partspec->strategy, "range")) *strategy = PARTITION_STRATEGY_RANGE; + else if (!pg_strcasecmp(partspec->strategy, "hash")) + *strategy = PARTITION_STRATEGY_HASH; else ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("unrecognized partitioning strategy \"%s\"", partspec->strategy)));+ if (*strategy == PARTITION_STRATEGY_HASH) + { + if (partspec->partnparts < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be specified for hash partition"))); + else if (partspec->partnparts == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be greater than 0"))); + + if (list_length(partspec->hashfunc) == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function must be specified for hash partition"))); + } + else + { + if (partspec->partnparts >= 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions can be specified only for hash partition"))); + + if (list_length(partspec->hashfunc) > 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function can be specified only for hash partition"))); + } + /* * Create a dummy ParseState and insert the target relation as its sole * rangetable entry. We need a ParseState for transformExpr. @@ -12843,7 +12905,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) * Compute per-partition-column information from a list of PartitionElem's */ static void -ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, +ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation) { int attn; @@ -13010,6 +13072,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, "btree", BTREE_AM_OID);+ partatttypes[attn] = atttype; attn++; } } diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c index 05d8538..f4febc9 100644 --- a/src/backend/nodes/copyfuncs.c +++ b/src/backend/nodes/copyfuncs.c @@ -4232,6 +4232,8 @@ _copyPartitionSpec(const PartitionSpec *from)COPY_STRING_FIELD(strategy);
COPY_NODE_FIELD(partParams);
+ COPY_SCALAR_FIELD(partnparts);
+ COPY_NODE_FIELD(hashfunc);
COPY_LOCATION_FIELD(location);return newnode; diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c index d595cd7..d589eac 100644 --- a/src/backend/nodes/equalfuncs.c +++ b/src/backend/nodes/equalfuncs.c @@ -2725,6 +2725,8 @@ _equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b) { COMPARE_STRING_FIELD(strategy); COMPARE_NODE_FIELD(partParams); + COMPARE_SCALAR_FIELD(partnparts); + COMPARE_NODE_FIELD(hashfunc); COMPARE_LOCATION_FIELD(location);return true; diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index b3802b4..d6db80e 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -3318,6 +3318,8 @@ _outPartitionSpec(StringInfo str, const PartitionSpec *node)WRITE_STRING_FIELD(strategy);
WRITE_NODE_FIELD(partParams);
+ WRITE_INT_FIELD(partnparts);
+ WRITE_NODE_FIELD(hashfunc);
WRITE_LOCATION_FIELD(location);
}diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index e833b2e..b67140d 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -574,6 +574,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); %type <list> partbound_datum_list %type <partrange_datum> PartitionRangeDatum %type <list> range_datum_list +%type <ival> hash_partitions +%type <list> hash_function/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -627,7 +629,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING
- HANDLER HAVING HEADER_P HOLD HOUR_P + HANDLER HASH HAVING HEADER_P HOLD HOUR_PIDENTITY_P IF_P ILIKE IMMEDIATE IMMUTABLE IMPLICIT_P IMPORT_P IN_P
INCLUDING INCREMENT INDEX INDEXES INHERIT INHERITS INITIALLY INLINE_P
@@ -651,7 +653,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
OBJECT_P OF OFF OFFSET OIDS OLD ON ONLY OPERATOR OPTION OPTIONS OR
ORDER ORDINALITY OUT_P OUTER_P OVER OVERLAPS OVERLAY OWNED OWNER- PARALLEL PARSER PARTIAL PARTITION PASSING PASSWORD PLACING PLANS POLICY + PARALLEL PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PLACING PLANS POLICY POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROGRAM PUBLICATION@@ -2587,6 +2589,16 @@ ForValues:
$$ = (Node *) n; } + + /* a HASH partition */ + | /*EMPTY*/ + { + PartitionBoundSpec *n = makeNode(PartitionBoundSpec); + + n->strategy = PARTITION_STRATEGY_HASH; + + $$ = (Node *) n; + } ;partbound_datum:
@@ -3666,7 +3678,7 @@ OptPartitionSpec: PartitionSpec { $$ = $1; }
| /*EMPTY*/ { $$ = NULL; }
;-PartitionSpec: PARTITION BY part_strategy '(' part_params ')' +PartitionSpec: PARTITION BY part_strategy '(' part_params ')' hash_partitions hash_function { PartitionSpec *n = makeNode(PartitionSpec);@@ -3674,10 +3686,21 @@ PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
n->partParams = $5;
n->location = @1;+ n->partnparts = $7; + n->hashfunc = $8; + $$ = n; } ;+hash_partitions: PARTITIONS Iconst { $$ = $2; } + | /*EMPTY*/ { $$ = -1; } + ; + +hash_function: USING handler_name { $$ = $2; } + | /*EMPTY*/ { $$ = NULL; } + ; + part_strategy: IDENT { $$ = $1; } | unreserved_keyword { $$ = pstrdup($1); } ; @@ -14377,6 +14400,7 @@ unreserved_keyword: | GLOBAL | GRANTED | HANDLER + | HASH | HEADER_P | HOLD | HOUR_P @@ -14448,6 +14472,7 @@ unreserved_keyword: | PARSER | PARTIAL | PARTITION + | PARTITIONS | PASSING | PASSWORD | PLANS diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c index ff2bab6..8e1be31 100644 --- a/src/backend/parser/parse_utilcmd.c +++ b/src/backend/parser/parse_utilcmd.c @@ -40,6 +40,7 @@ #include "catalog/pg_opclass.h" #include "catalog/pg_operator.h" #include "catalog/pg_type.h" +#include "catalog/partition.h" #include "commands/comment.h" #include "commands/defrem.h" #include "commands/tablecmds.h" @@ -3252,6 +3253,24 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound) ++i; } } + else if (strategy == PARTITION_STRATEGY_HASH) + { + Value *conval; + Node *value; + int index; + + if (spec->strategy != PARTITION_STRATEGY_HASH) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("invalid bound specification for a hash partition"))); + + index = get_next_hash_partition_index(parent); + + /* store the partition index as a listdatums value */ + conval = makeInteger(index); + value = (Node *) make_const(pstate, conval, -1); + result_spec->listdatums = list_make1(value); + } else elog(ERROR, "unexpected partition strategy: %d", (int) strategy);diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c index b27b77d..fab6eea 100644 --- a/src/backend/utils/adt/ruleutils.c +++ b/src/backend/utils/adt/ruleutils.c @@ -1423,7 +1423,7 @@ pg_get_indexdef_worker(Oid indexrelid, int colno, * * 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) @@ -1513,6 +1513,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags) case PARTITION_STRATEGY_RANGE: appendStringInfo(&buf, "RANGE"); break; + case PARTITION_STRATEGY_HASH: + appendStringInfo(&buf, "HASH"); + break; default: elog(ERROR, "unexpected partition strategy: %d", (int) form->partstrat); @@ -8520,6 +8523,9 @@ get_rule_expr(Node *node, deparse_context *context, appendStringInfoString(buf, ")"); break;+ case PARTITION_STRATEGY_HASH: + break; + default: elog(ERROR, "unrecognized partition strategy: %d", (int) spec->strategy); diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c index 9001e20..829e4d2 100644 --- a/src/backend/utils/cache/relcache.c +++ b/src/backend/utils/cache/relcache.c @@ -855,6 +855,9 @@ RelationBuildPartitionKey(Relation relation) key->strategy = form->partstrat; key->partnatts = form->partnatts;+ key->partnparts = form->partnparts; + key->parthashfunc = form->parthashfunc; + /* * We can rely on the first variable-length attribute being mapped to the * relevant field of the catalog's C struct, because all previous @@ -999,6 +1002,9 @@ copy_partition_key(PartitionKey fromkey) newkey->strategy = fromkey->strategy; newkey->partnatts = n = fromkey->partnatts;+ newkey->partnparts = fromkey->partnparts; + newkey->parthashfunc = fromkey->parthashfunc; + newkey->partattrs = (AttrNumber *) palloc(n * sizeof(AttrNumber)); memcpy(newkey->partattrs, fromkey->partattrs, n * sizeof(AttrNumber));diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h index 1187797..367e2f8 100644 --- a/src/include/catalog/heap.h +++ b/src/include/catalog/heap.h @@ -141,7 +141,7 @@ extern void StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation); + Oid *partcollation, int16 partnparts, Oid hashfunc); extern void RemovePartitionKeyByRelId(Oid relid); extern void StorePartitionBound(Relation rel, Relation parent, Node *bound);diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h index b195d1a..80f4b0e 100644 --- a/src/include/catalog/partition.h +++ b/src/include/catalog/partition.h @@ -89,4 +89,6 @@ extern int get_partition_for_tuple(PartitionDispatch *pd, TupleTableSlot *slot, EState *estate, Oid *failed_at); +extern Expr *convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc); +extern int get_next_hash_partition_index(Relation parent); #endif /* PARTITION_H */ diff --git a/src/include/catalog/pg_partitioned_table.h b/src/include/catalog/pg_partitioned_table.h index bdff36a..69e509c 100644 --- a/src/include/catalog/pg_partitioned_table.h +++ b/src/include/catalog/pg_partitioned_table.h @@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS char partstrat; /* partitioning strategy */ int16 partnatts; /* number of partition key columns */+ int16 partnparts;
+ Oid parthashfunc;
+
/*
* variable-length fields start here, but we allow direct access to
* partattrs via the C struct. That's because the first variable-length
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;/* ---------------- @@ -62,13 +67,15 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table; * compiler constants for pg_partitioned_table * ---------------- */ -#define Natts_pg_partitioned_table 7 +#define Natts_pg_partitioned_table 9 #define Anum_pg_partitioned_table_partrelid 1 #define Anum_pg_partitioned_table_partstrat 2 #define Anum_pg_partitioned_table_partnatts 3 -#define Anum_pg_partitioned_table_partattrs 4 -#define Anum_pg_partitioned_table_partclass 5 -#define Anum_pg_partitioned_table_partcollation 6 -#define Anum_pg_partitioned_table_partexprs 7 +#define Anum_pg_partitioned_table_partnparts 4 +#define Anum_pg_partitioned_table_parthashfunc 5 +#define Anum_pg_partitioned_table_partattrs 6 +#define Anum_pg_partitioned_table_partclass 7 +#define Anum_pg_partitioned_table_partcollation 8 +#define Anum_pg_partitioned_table_partexprs 9#endif /* PG_PARTITIONED_TABLE_H */ diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index 5afc3eb..1c3474f 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -730,11 +730,14 @@ typedef struct PartitionSpec NodeTag type; char *strategy; /* partitioning strategy ('list' or 'range') */ List *partParams; /* List of PartitionElems */ + int partnparts; + List *hashfunc; int location; /* token location, or -1 if unknown */ } PartitionSpec;#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'/* * PartitionBoundSpec - a partition bound specification diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index 985d650..0597939 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -180,6 +180,7 @@ PG_KEYWORD("greatest", GREATEST, COL_NAME_KEYWORD) PG_KEYWORD("group", GROUP_P, RESERVED_KEYWORD) PG_KEYWORD("grouping", GROUPING, COL_NAME_KEYWORD) PG_KEYWORD("handler", HANDLER, UNRESERVED_KEYWORD) +PG_KEYWORD("hash", HASH, UNRESERVED_KEYWORD) PG_KEYWORD("having", HAVING, RESERVED_KEYWORD) PG_KEYWORD("header", HEADER_P, UNRESERVED_KEYWORD) PG_KEYWORD("hold", HOLD, UNRESERVED_KEYWORD) @@ -291,6 +292,7 @@ PG_KEYWORD("parallel", PARALLEL, UNRESERVED_KEYWORD) PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD) PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD) PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD) +PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD) PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD) PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD) PG_KEYWORD("placing", PLACING, RESERVED_KEYWORD) diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h index a617a7c..660adfb 100644 --- a/src/include/utils/rel.h +++ b/src/include/utils/rel.h @@ -62,6 +62,9 @@ typedef struct PartitionKeyData Oid *partopcintype; /* OIDs of opclass declared input data types */ FmgrInfo *partsupfunc; /* lookup info for support funcs */+ int16 partnparts; /* number of hash partitions */ + Oid parthashfunc; /* OID of hash function */ + /* Partitioning collation per attribute */ Oid *partcollation;
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
--
Best regards,
Aleksander Alekseev
Nagata-san,
On 2017/02/28 23:33, Yugo Nagata wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.
Great!
* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for h
Instead of having to create each partition individually, wouldn't it be
better if the following command
CREATE TABLE h (i int) PARTITION BY HASH (i) PARTITIONS 3;
created the partitions *automatically*?
It makes sense to provide a way to create individual list and range
partitions separately, because users can specify custom bounds for each.
We don't need that for hash partitions, so why make users run separate
commands (without the FOR VALUES clause) anyway? We may perhaps need to
offer a way to optionally specify a user-defined name for each partition
in the same command, along with tablespace, storage options, etc. By
default, the names would be generated internally and the user can ALTER
individual partitions after the fact to specify tablespace, etc.
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 Tue, Feb 28, 2017 at 8:03 PM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)
This is good, I will have closer look into the patch, but here are
few quick comments.
- CREATE HASH partition syntax adds two new keywords and ideally
we should try to avoid adding additional keywords. Also I can see that
HASH keyword been added, but I don't see any use of newly added
keyword in gram.y.
- Also I didn't like the idea of fixing number of partitions during the
CREATE
TABLE syntax. Thats something that needs to be able to changes.
* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.
+1
- With fixing default hash function and not specifying number of partitions
during CREATE TABLE - don't need two new additional columns into
pg_partitioned_table catalog.
Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().postgres=# explain select * from h where abs(hashint4(i))%3 =
abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)Best regards,
Yugo Nagata--
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
Regards,
Rushabh Lathia
On Tue, Feb 28, 2017 at 8:03 PM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.
Great.
* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed
in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.
IMHO, we should try to keep create partition syntax simple and aligned
with other partition strategy. For e.g:
CREATE TABLE h (i int) PARTITION BY HASH(i);
I Agree that it is unavoidable partitions number in modulo hashing,
but we can do in other hashing technique. Have you had thought about
Linear hashing[1]https://en.wikipedia.org/wiki/Linear_hashing or Consistent hashing[2]https://en.wikipedia.org/wiki/Consistent_hashing? This will allow us to
add/drop
partition with minimal row moment.
+1 for the default hash function corresponding to partitioning key type.
Regards,
Amul
[1]: https://en.wikipedia.org/wiki/Linear_hashing
[2]: https://en.wikipedia.org/wiki/Consistent_hashing
Hi Aleksander,
On Tue, 28 Feb 2017 18:05:36 +0300
Aleksander Alekseev <a.alekseev@postgrespro.ru> wrote:
Hi, Yugo.
Looks like a great feature! I'm going to take a closer look on your code
and write a feedback shortly. For now I can only tell that you forgot
to include some documentation in the patch.
Thank you for looking into it. I'm forward to your feedback.
This is a proof of concept patch and additional documentation
is not included. I'll add this after reaching a consensus
on the specification of the feature.
I've added a corresponding entry to current commitfest [1]. Hope you
don't mind. If it's not too much trouble could you please register on a
commitfest site and add yourself to this entry as an author? I'm pretty
sure someone is using this information for writing release notes or
something like this.
Thank you for registering it to the commitfest. I have added me as an auther.
[1] https://commitfest.postgresql.org/13/1059/
On Tue, Feb 28, 2017 at 11:33:13PM +0900, Yugo Nagata wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().postgres=# explain select * from h where abs(hashint4(i))%3 = abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)Best regards,
Yugo Nagata--
Yugo Nagata <nagata@sraoss.co.jp>diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c index 41c0056..3820920 100644 --- a/src/backend/catalog/heap.c +++ b/src/backend/catalog/heap.c @@ -3074,7 +3074,7 @@ StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation) + Oid *partcollation, int16 partnparts, Oid hashfunc) { int i; int2vector *partattrs_vec; @@ -3121,6 +3121,8 @@ StorePartitionKey(Relation rel, values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel)); values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy); values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts); + values[Anum_pg_partitioned_table_partnparts - 1] = Int16GetDatum(partnparts); + values[Anum_pg_partitioned_table_parthashfunc - 1] = ObjectIdGetDatum(hashfunc); values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec); values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec); values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec); diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c index 4bcef58..24e69c6 100644 --- a/src/backend/catalog/partition.c +++ b/src/backend/catalog/partition.c @@ -36,6 +36,8 @@ #include "optimizer/clauses.h" #include "optimizer/planmain.h" #include "optimizer/var.h" +#include "parser/parse_func.h" +#include "parser/parse_oper.h" #include "rewrite/rewriteManip.h" #include "storage/lmgr.h" #include "utils/array.h" @@ -120,6 +122,7 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec); static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec); +static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec); static Oid get_partition_operator(PartitionKey key, int col, StrategyNumber strategy, bool *need_relabel); static List *generate_partition_qual(Relation rel); @@ -236,7 +239,8 @@ 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_LIST || + key->strategy == PARTITION_STRATEGY_HASH) { List *non_null_values = NIL;@@ -251,7 +255,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);- if (spec->strategy != PARTITION_STRATEGY_LIST) + if (spec->strategy != key->strategy) elog(ERROR, "invalid strategy in partition bound spec");foreach(c, spec->listdatums)
@@ -464,6 +468,7 @@ RelationBuildPartitionDesc(Relation rel)
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
{
boundinfo->has_null = found_null;
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
@@ -829,6 +834,18 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
break;
}+ case PARTITION_STRATEGY_HASH: + { + Assert(spec->strategy == PARTITION_STRATEGY_HASH); + + if (partdesc->nparts + 1 > key->partnparts) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("cannot create hash partition more than %d for %s", + key->partnparts, RelationGetRelationName(parent)))); + break; + } + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -916,6 +933,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound) my_qual = get_qual_for_range(key, spec); break;+ case PARTITION_STRATEGY_HASH: + Assert(spec->strategy == PARTITION_STRATEGY_LIST); + my_qual = get_qual_for_hash(key, spec); + break; + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -1146,6 +1168,84 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode, return pd; }+/* + * convert_expr_for_hash + * + * Converts a expr for a hash partition's constraint. + * expr is converted into 'abs(hashfunc(expr)) % npart". + * + * npart: number of partitions + * hashfunc: OID of hash function + */ +Expr * +convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc) +{ + FuncExpr *func, + *abs; + Expr *modexpr; + Oid modoid; + Oid int4oid[1] = {INT4OID}; + + ParseState *pstate = make_parsestate(NULL); + Value *val_npart = makeInteger(npart); + Node *const_npart = (Node *) make_const(pstate, val_npart, -1); + + /* hash function */ + func = makeFuncExpr(hashfunc, + INT4OID, + list_make1(expr), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* Abs */ + abs = makeFuncExpr(LookupFuncName(list_make1(makeString("abs")), 1, int4oid, false), + INT4OID, + list_make1(func), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* modulo by npart */ + modoid = LookupOperName(pstate, list_make1(makeString("%")), INT4OID, INT4OID, false, -1); + modexpr = make_opclause(modoid, INT4OID, false, (Expr*)abs, (Expr*)const_npart, 0, 0); + + return modexpr; +} + + +/* + * get_next_hash_partition_index + * + * Returns the minimal index which is not used for hash partition. + */ +int +get_next_hash_partition_index(Relation parent) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionDesc partdesc = RelationGetPartitionDesc(parent); + + int i; + bool *used = palloc0(sizeof(int) * key->partnparts); + + /* mark used for existing partition indexs */ + for (i = 0; i < partdesc->boundinfo->ndatums; i++) + { + Datum* datum = partdesc->boundinfo->datums[i]; + int idx = DatumGetInt16(datum[0]); + + if (!used[idx]) + used[idx] = true; + } + + /* find the minimal unused index */ + for (i = 0; i < key->partnparts; i++) + if (!used[i]) + break; + + return i; +} + /* Module-local functions *//*
@@ -1467,6 +1567,43 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
}/* + * get_qual_for_hash + * + * Returns a list of expressions to use as a hash partition's constraint. + */ +static List * +get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec) +{ + List *result; + Expr *keyCol; + Expr *expr; + Expr *opexpr; + Oid operoid; + ParseState *pstate = make_parsestate(NULL); + + /* Left operand */ + if (key->partattrs[0] != 0) + keyCol = (Expr *) makeVar(1, + key->partattrs[0], + key->parttypid[0], + key->parttypmod[0], + key->parttypcoll[0], + 0); + else + keyCol = (Expr *) copyObject(linitial(key->partexprs)); + + expr = convert_expr_for_hash(keyCol, key->partnparts, key->parthashfunc); + + /* equals the listdaums value */ + operoid = LookupOperName(pstate, list_make1(makeString("=")), INT4OID, INT4OID, false, -1); + opexpr = make_opclause(operoid, BOOLOID, false, expr, linitial(spec->listdatums), 0, 0); + + result = list_make1(opexpr); + + return result; +} + +/* * get_partition_operator * * Return oid of the operator of given strategy for a given partition key @@ -1730,6 +1867,11 @@ get_partition_for_tuple(PartitionDispatch *pd, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), errmsg("range partition key of row contains null"))); } + else if (key->strategy == PARTITION_STRATEGY_HASH) + { + values[0] = OidFunctionCall1(key->parthashfunc, values[0]); + values[0] = Int16GetDatum(Abs(DatumGetInt16(values[0])) % key->partnparts); + }if (partdesc->boundinfo->has_null && isnull[0]) /* Tuple maps to the null-accepting list partition */ @@ -1744,6 +1886,7 @@ get_partition_for_tuple(PartitionDispatch *pd, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: if (cur_offset >= 0 && equal) cur_index = partdesc->boundinfo->indexes[cur_offset]; else @@ -1968,6 +2111,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0], key->partcollation[0], bound_datums[0], diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index 3cea220..5a28cc0 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -41,6 +41,7 @@ #include "catalog/pg_inherits_fn.h" #include "catalog/pg_namespace.h" #include "catalog/pg_opclass.h" +#include "catalog/pg_proc.h" #include "catalog/pg_tablespace.h" #include "catalog/pg_trigger.h" #include "catalog/pg_type.h" @@ -77,6 +78,7 @@ #include "parser/parse_oper.h" #include "parser/parse_relation.h" #include "parser/parse_type.h" +#include "parser/parse_func.h" #include "parser/parse_utilcmd.h" #include "parser/parser.h" #include "pgstat.h" @@ -450,7 +452,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid, Oid oldrelid, void *arg); 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, +static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation); static void CreateInheritance(Relation child_rel, Relation parent_rel); static void RemoveInheritance(Relation child_rel, Relation parent_rel); @@ -799,8 +801,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, AttrNumber partattrs[PARTITION_MAX_KEYS]; Oid partopclass[PARTITION_MAX_KEYS]; Oid partcollation[PARTITION_MAX_KEYS]; + Oid partatttypes[PARTITION_MAX_KEYS]; List *partexprs = NIL; List *cmds = NIL; + Oid hashfuncOid = InvalidOid;/* * We need to transform the raw parsetrees corresponding to partition @@ -811,15 +815,40 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, stmt->partspec = transformPartitionSpec(rel, stmt->partspec, &strategy); ComputePartitionAttrs(rel, stmt->partspec->partParams, - partattrs, &partexprs, partopclass, + partattrs, partatttypes, &partexprs, partopclass, partcollation);partnatts = list_length(stmt->partspec->partParams); + + if (strategy == PARTITION_STRATEGY_HASH) + { + Oid funcrettype; + + if (partnatts != 1) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partition key must be 1 for hash partition"))); + + hashfuncOid = LookupFuncName(stmt->partspec->hashfunc, 1, partatttypes, false); + funcrettype = get_func_rettype(hashfuncOid); + if (funcrettype != INT4OID) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must return integer"))); + + if (func_volatile(hashfuncOid) != PROVOLATILE_IMMUTABLE) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must be marked IMMUTABLE"))); + + } + StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs, - partopclass, partcollation); + partopclass, partcollation, stmt->partspec->partnparts, hashfuncOid);- /* Force key columns to be NOT NULL when using range partitioning */ - if (strategy == PARTITION_STRATEGY_RANGE) + /* Force key columns to be NOT NULL when using range or hash partitioning */ + if (strategy == PARTITION_STRATEGY_RANGE || + strategy == PARTITION_STRATEGY_HASH) { for (i = 0; i < partnatts; i++) { @@ -12783,18 +12812,51 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) newspec->strategy = partspec->strategy; newspec->location = partspec->location; newspec->partParams = NIL; + newspec->partnparts = partspec->partnparts; + newspec->hashfunc = partspec->hashfunc;/* Parse partitioning strategy name */ if (!pg_strcasecmp(partspec->strategy, "list")) *strategy = PARTITION_STRATEGY_LIST; else if (!pg_strcasecmp(partspec->strategy, "range")) *strategy = PARTITION_STRATEGY_RANGE; + else if (!pg_strcasecmp(partspec->strategy, "hash")) + *strategy = PARTITION_STRATEGY_HASH; else ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("unrecognized partitioning strategy \"%s\"", partspec->strategy)));+ if (*strategy == PARTITION_STRATEGY_HASH) + { + if (partspec->partnparts < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be specified for hash partition"))); + else if (partspec->partnparts == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be greater than 0"))); + + if (list_length(partspec->hashfunc) == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function must be specified for hash partition"))); + } + else + { + if (partspec->partnparts >= 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions can be specified only for hash partition"))); + + if (list_length(partspec->hashfunc) > 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function can be specified only for hash partition"))); + } + /* * Create a dummy ParseState and insert the target relation as its sole * rangetable entry. We need a ParseState for transformExpr. @@ -12843,7 +12905,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) * Compute per-partition-column information from a list of PartitionElem's */ static void -ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, +ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation) { int attn; @@ -13010,6 +13072,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, "btree", BTREE_AM_OID);+ partatttypes[attn] = atttype; attn++; } } diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c index 05d8538..f4febc9 100644 --- a/src/backend/nodes/copyfuncs.c +++ b/src/backend/nodes/copyfuncs.c @@ -4232,6 +4232,8 @@ _copyPartitionSpec(const PartitionSpec *from)COPY_STRING_FIELD(strategy);
COPY_NODE_FIELD(partParams);
+ COPY_SCALAR_FIELD(partnparts);
+ COPY_NODE_FIELD(hashfunc);
COPY_LOCATION_FIELD(location);return newnode; diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c index d595cd7..d589eac 100644 --- a/src/backend/nodes/equalfuncs.c +++ b/src/backend/nodes/equalfuncs.c @@ -2725,6 +2725,8 @@ _equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b) { COMPARE_STRING_FIELD(strategy); COMPARE_NODE_FIELD(partParams); + COMPARE_SCALAR_FIELD(partnparts); + COMPARE_NODE_FIELD(hashfunc); COMPARE_LOCATION_FIELD(location);return true; diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index b3802b4..d6db80e 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -3318,6 +3318,8 @@ _outPartitionSpec(StringInfo str, const PartitionSpec *node)WRITE_STRING_FIELD(strategy);
WRITE_NODE_FIELD(partParams);
+ WRITE_INT_FIELD(partnparts);
+ WRITE_NODE_FIELD(hashfunc);
WRITE_LOCATION_FIELD(location);
}diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index e833b2e..b67140d 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -574,6 +574,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); %type <list> partbound_datum_list %type <partrange_datum> PartitionRangeDatum %type <list> range_datum_list +%type <ival> hash_partitions +%type <list> hash_function/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -627,7 +629,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING
- HANDLER HAVING HEADER_P HOLD HOUR_P + HANDLER HASH HAVING HEADER_P HOLD HOUR_PIDENTITY_P IF_P ILIKE IMMEDIATE IMMUTABLE IMPLICIT_P IMPORT_P IN_P
INCLUDING INCREMENT INDEX INDEXES INHERIT INHERITS INITIALLY INLINE_P
@@ -651,7 +653,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
OBJECT_P OF OFF OFFSET OIDS OLD ON ONLY OPERATOR OPTION OPTIONS OR
ORDER ORDINALITY OUT_P OUTER_P OVER OVERLAPS OVERLAY OWNED OWNER- PARALLEL PARSER PARTIAL PARTITION PASSING PASSWORD PLACING PLANS POLICY + PARALLEL PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PLACING PLANS POLICY POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROGRAM PUBLICATION@@ -2587,6 +2589,16 @@ ForValues:
$$ = (Node *) n; } + + /* a HASH partition */ + | /*EMPTY*/ + { + PartitionBoundSpec *n = makeNode(PartitionBoundSpec); + + n->strategy = PARTITION_STRATEGY_HASH; + + $$ = (Node *) n; + } ;partbound_datum:
@@ -3666,7 +3678,7 @@ OptPartitionSpec: PartitionSpec { $$ = $1; }
| /*EMPTY*/ { $$ = NULL; }
;-PartitionSpec: PARTITION BY part_strategy '(' part_params ')' +PartitionSpec: PARTITION BY part_strategy '(' part_params ')' hash_partitions hash_function { PartitionSpec *n = makeNode(PartitionSpec);@@ -3674,10 +3686,21 @@ PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
n->partParams = $5;
n->location = @1;+ n->partnparts = $7; + n->hashfunc = $8; + $$ = n; } ;+hash_partitions: PARTITIONS Iconst { $$ = $2; } + | /*EMPTY*/ { $$ = -1; } + ; + +hash_function: USING handler_name { $$ = $2; } + | /*EMPTY*/ { $$ = NULL; } + ; + part_strategy: IDENT { $$ = $1; } | unreserved_keyword { $$ = pstrdup($1); } ; @@ -14377,6 +14400,7 @@ unreserved_keyword: | GLOBAL | GRANTED | HANDLER + | HASH | HEADER_P | HOLD | HOUR_P @@ -14448,6 +14472,7 @@ unreserved_keyword: | PARSER | PARTIAL | PARTITION + | PARTITIONS | PASSING | PASSWORD | PLANS diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c index ff2bab6..8e1be31 100644 --- a/src/backend/parser/parse_utilcmd.c +++ b/src/backend/parser/parse_utilcmd.c @@ -40,6 +40,7 @@ #include "catalog/pg_opclass.h" #include "catalog/pg_operator.h" #include "catalog/pg_type.h" +#include "catalog/partition.h" #include "commands/comment.h" #include "commands/defrem.h" #include "commands/tablecmds.h" @@ -3252,6 +3253,24 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound) ++i; } } + else if (strategy == PARTITION_STRATEGY_HASH) + { + Value *conval; + Node *value; + int index; + + if (spec->strategy != PARTITION_STRATEGY_HASH) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("invalid bound specification for a hash partition"))); + + index = get_next_hash_partition_index(parent); + + /* store the partition index as a listdatums value */ + conval = makeInteger(index); + value = (Node *) make_const(pstate, conval, -1); + result_spec->listdatums = list_make1(value); + } else elog(ERROR, "unexpected partition strategy: %d", (int) strategy);diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c index b27b77d..fab6eea 100644 --- a/src/backend/utils/adt/ruleutils.c +++ b/src/backend/utils/adt/ruleutils.c @@ -1423,7 +1423,7 @@ pg_get_indexdef_worker(Oid indexrelid, int colno, * * 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) @@ -1513,6 +1513,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags) case PARTITION_STRATEGY_RANGE: appendStringInfo(&buf, "RANGE"); break; + case PARTITION_STRATEGY_HASH: + appendStringInfo(&buf, "HASH"); + break; default: elog(ERROR, "unexpected partition strategy: %d", (int) form->partstrat); @@ -8520,6 +8523,9 @@ get_rule_expr(Node *node, deparse_context *context, appendStringInfoString(buf, ")"); break;+ case PARTITION_STRATEGY_HASH: + break; + default: elog(ERROR, "unrecognized partition strategy: %d", (int) spec->strategy); diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c index 9001e20..829e4d2 100644 --- a/src/backend/utils/cache/relcache.c +++ b/src/backend/utils/cache/relcache.c @@ -855,6 +855,9 @@ RelationBuildPartitionKey(Relation relation) key->strategy = form->partstrat; key->partnatts = form->partnatts;+ key->partnparts = form->partnparts; + key->parthashfunc = form->parthashfunc; + /* * We can rely on the first variable-length attribute being mapped to the * relevant field of the catalog's C struct, because all previous @@ -999,6 +1002,9 @@ copy_partition_key(PartitionKey fromkey) newkey->strategy = fromkey->strategy; newkey->partnatts = n = fromkey->partnatts;+ newkey->partnparts = fromkey->partnparts; + newkey->parthashfunc = fromkey->parthashfunc; + newkey->partattrs = (AttrNumber *) palloc(n * sizeof(AttrNumber)); memcpy(newkey->partattrs, fromkey->partattrs, n * sizeof(AttrNumber));diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h index 1187797..367e2f8 100644 --- a/src/include/catalog/heap.h +++ b/src/include/catalog/heap.h @@ -141,7 +141,7 @@ extern void StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation); + Oid *partcollation, int16 partnparts, Oid hashfunc); extern void RemovePartitionKeyByRelId(Oid relid); extern void StorePartitionBound(Relation rel, Relation parent, Node *bound);diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h index b195d1a..80f4b0e 100644 --- a/src/include/catalog/partition.h +++ b/src/include/catalog/partition.h @@ -89,4 +89,6 @@ extern int get_partition_for_tuple(PartitionDispatch *pd, TupleTableSlot *slot, EState *estate, Oid *failed_at); +extern Expr *convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc); +extern int get_next_hash_partition_index(Relation parent); #endif /* PARTITION_H */ diff --git a/src/include/catalog/pg_partitioned_table.h b/src/include/catalog/pg_partitioned_table.h index bdff36a..69e509c 100644 --- a/src/include/catalog/pg_partitioned_table.h +++ b/src/include/catalog/pg_partitioned_table.h @@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS char partstrat; /* partitioning strategy */ int16 partnatts; /* number of partition key columns */+ int16 partnparts;
+ Oid parthashfunc;
+
/*
* variable-length fields start here, but we allow direct access to
* partattrs via the C struct. That's because the first variable-length
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;/* ---------------- @@ -62,13 +67,15 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table; * compiler constants for pg_partitioned_table * ---------------- */ -#define Natts_pg_partitioned_table 7 +#define Natts_pg_partitioned_table 9 #define Anum_pg_partitioned_table_partrelid 1 #define Anum_pg_partitioned_table_partstrat 2 #define Anum_pg_partitioned_table_partnatts 3 -#define Anum_pg_partitioned_table_partattrs 4 -#define Anum_pg_partitioned_table_partclass 5 -#define Anum_pg_partitioned_table_partcollation 6 -#define Anum_pg_partitioned_table_partexprs 7 +#define Anum_pg_partitioned_table_partnparts 4 +#define Anum_pg_partitioned_table_parthashfunc 5 +#define Anum_pg_partitioned_table_partattrs 6 +#define Anum_pg_partitioned_table_partclass 7 +#define Anum_pg_partitioned_table_partcollation 8 +#define Anum_pg_partitioned_table_partexprs 9#endif /* PG_PARTITIONED_TABLE_H */ diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index 5afc3eb..1c3474f 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -730,11 +730,14 @@ typedef struct PartitionSpec NodeTag type; char *strategy; /* partitioning strategy ('list' or 'range') */ List *partParams; /* List of PartitionElems */ + int partnparts; + List *hashfunc; int location; /* token location, or -1 if unknown */ } PartitionSpec;#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'/* * PartitionBoundSpec - a partition bound specification diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index 985d650..0597939 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -180,6 +180,7 @@ PG_KEYWORD("greatest", GREATEST, COL_NAME_KEYWORD) PG_KEYWORD("group", GROUP_P, RESERVED_KEYWORD) PG_KEYWORD("grouping", GROUPING, COL_NAME_KEYWORD) PG_KEYWORD("handler", HANDLER, UNRESERVED_KEYWORD) +PG_KEYWORD("hash", HASH, UNRESERVED_KEYWORD) PG_KEYWORD("having", HAVING, RESERVED_KEYWORD) PG_KEYWORD("header", HEADER_P, UNRESERVED_KEYWORD) PG_KEYWORD("hold", HOLD, UNRESERVED_KEYWORD) @@ -291,6 +292,7 @@ PG_KEYWORD("parallel", PARALLEL, UNRESERVED_KEYWORD) PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD) PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD) PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD) +PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD) PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD) PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD) PG_KEYWORD("placing", PLACING, RESERVED_KEYWORD) diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h index a617a7c..660adfb 100644 --- a/src/include/utils/rel.h +++ b/src/include/utils/rel.h @@ -62,6 +62,9 @@ typedef struct PartitionKeyData Oid *partopcintype; /* OIDs of opclass declared input data types */ FmgrInfo *partsupfunc; /* lookup info for support funcs */+ int16 partnparts; /* number of hash partitions */ + Oid parthashfunc; /* OID of hash function */ + /* Partitioning collation per attribute */ Oid *partcollation;--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers--
Best regards,
Aleksander Alekseev
--
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
Hi Ammit,
On Wed, 1 Mar 2017 11:14:15 +0900
Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:
Nagata-san,
On 2017/02/28 23:33, Yugo Nagata wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.Great!
Thank you!
* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hInstead of having to create each partition individually, wouldn't it be
better if the following commandCREATE TABLE h (i int) PARTITION BY HASH (i) PARTITIONS 3;
created the partitions *automatically*?
It makes sense to provide a way to create individual list and range
partitions separately, because users can specify custom bounds for each.
We don't need that for hash partitions, so why make users run separate
commands (without the FOR VALUES clause) anyway? We may perhaps need to
offer a way to optionally specify a user-defined name for each partition
in the same command, along with tablespace, storage options, etc. By
default, the names would be generated internally and the user can ALTER
individual partitions after the fact to specify tablespace, etc.
I though that creating each partition individually is needed because some
user will want to specify a tablespce to each partition. However, as you
say, that isn't need for many cases because use can move a partition
to other tablespaces afterward by ALTER.
Thanks,
Yugo Nagata
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
--
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
On Wed, 1 Mar 2017 10:30:09 +0530
Rushabh Lathia <rushabh.lathia@gmail.com> wrote:
On Tue, Feb 28, 2017 at 8:03 PM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)This is good, I will have closer look into the patch, but here are
few quick comments.
Thanks. I'm looking forward to your comments.
- CREATE HASH partition syntax adds two new keywords and ideally
we should try to avoid adding additional keywords. Also I can see that
HASH keyword been added, but I don't see any use of newly added
keyword in gram.y.
Yes, you are right. HASH keyword is not necessary. I'll remove it
from the patch.
- Also I didn't like the idea of fixing number of partitions during the
CREATE
TABLE syntax. Thats something that needs to be able to changes.
I agree. The number specified by PARTIONS should be the *initial* number
of partitions and this should be abelt to be changed. I'm investigating
the way.
* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.+1
- With fixing default hash function and not specifying number of partitions
during CREATE TABLE - don't need two new additional columns into
pg_partitioned_table catalog.
I think the option to specify a hash function is needed because
user may want to use a user-defined hash function for some reasons,
for example, when a user-defined type is used as a partition key.
Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().postgres=# explain select * from h where abs(hashint4(i))%3 =
abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)Best regards,
Yugo Nagata--
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-hackersRegards,
Rushabh Lathia
--
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
On Wed, 1 Mar 2017 10:52:58 +0530
amul sul <sulamul@gmail.com> wrote:
On Tue, Feb 28, 2017 at 8:03 PM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.Great.
Thanks.
* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changedin
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.IMHO, we should try to keep create partition syntax simple and aligned
with other partition strategy. For e.g:
CREATE TABLE h (i int) PARTITION BY HASH(i);I Agree that it is unavoidable partitions number in modulo hashing,
but we can do in other hashing technique. Have you had thought about
Linear hashing[1] or Consistent hashing[2]? This will allow us to
add/drop
partition with minimal row moment.
Thank you for your information of hash technique. I'll see them
and try to allowing the number of partitions to be changed.
Thanks,
Yugo Nagata
+1 for the default hash function corresponding to partitioning key type.
Regards,
Amul
[1] https://en.wikipedia.org/wiki/Linear_hashing
[2] https://en.wikipedia.org/wiki/Consistent_hashing
--
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
Hi, Yugo.
Today I've had an opportunity to take a closer look on this patch. Here are
a few things that bother me.
1a) There are missing commends here:
```
--- a/src/include/catalog/pg_partitioned_table.h
+++ b/src/include/catalog/pg_partitioned_table.h
@@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
char partstrat; /* partitioning strategy */
int16 partnatts; /* number of partition key columns */
+ int16 partnparts;
+ Oid parthashfunc;
+
```
1b) ... and here:
```
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -730,11 +730,14 @@ typedef struct PartitionSpec
NodeTag type;
char *strategy; /* partitioning strategy ('list' or 'range') */
List *partParams; /* List of PartitionElems */
+ int partnparts;
+ List *hashfunc;
int location; /* token location, or -1 if unknown */
} PartitionSpec;
```
2) I believe new empty lines in patches are generally not welcomed by
community:
```
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;
```
3) One test fails on my laptop (Arch Linux, x64) [1]http://afiskon.ru/s/dd/20cbe21934_regression.diffs.txt:
```
***************
*** 344,350 ****
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
--- 344,350 ----
CREATE TABLE partitioned (
a int
) PARTITION BY HASH (a);
! ERROR: number of partitions must be specified for hash partition
-- specified column must be present in the table
CREATE TABLE partitioned (
a int
```
Exact script I'm using for building and testing PostgreSQL could be
found here [2]http://afiskon.ru/s/76/a4fb71739c_full-build.sh.txt.
4) As I already mentioned - missing documentation.
In general patch looks quite good to me. I personally believe it has all
the changes to be accepted in current commitfest. Naturally if community
will come to a consensus regarding keywords, whether all partitions
should be created automatically, etc :)
[1]: http://afiskon.ru/s/dd/20cbe21934_regression.diffs.txt
[2]: http://afiskon.ru/s/76/a4fb71739c_full-build.sh.txt
On Wed, Mar 01, 2017 at 06:10:10PM +0900, Yugo Nagata wrote:
Hi Aleksander,
On Tue, 28 Feb 2017 18:05:36 +0300
Aleksander Alekseev <a.alekseev@postgrespro.ru> wrote:Hi, Yugo.
Looks like a great feature! I'm going to take a closer look on your code
and write a feedback shortly. For now I can only tell that you forgot
to include some documentation in the patch.Thank you for looking into it. I'm forward to your feedback.
This is a proof of concept patch and additional documentation
is not included. I'll add this after reaching a consensus
on the specification of the feature.I've added a corresponding entry to current commitfest [1]. Hope you
don't mind. If it's not too much trouble could you please register on a
commitfest site and add yourself to this entry as an author? I'm pretty
sure someone is using this information for writing release notes or
something like this.Thank you for registering it to the commitfest. I have added me as an auther.
[1] https://commitfest.postgresql.org/13/1059/
On Tue, Feb 28, 2017 at 11:33:13PM +0900, Yugo Nagata wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().postgres=# explain select * from h where abs(hashint4(i))%3 = abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)Best regards,
Yugo Nagata--
Yugo Nagata <nagata@sraoss.co.jp>diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c index 41c0056..3820920 100644 --- a/src/backend/catalog/heap.c +++ b/src/backend/catalog/heap.c @@ -3074,7 +3074,7 @@ StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation) + Oid *partcollation, int16 partnparts, Oid hashfunc) { int i; int2vector *partattrs_vec; @@ -3121,6 +3121,8 @@ StorePartitionKey(Relation rel, values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel)); values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy); values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts); + values[Anum_pg_partitioned_table_partnparts - 1] = Int16GetDatum(partnparts); + values[Anum_pg_partitioned_table_parthashfunc - 1] = ObjectIdGetDatum(hashfunc); values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec); values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec); values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec); diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c index 4bcef58..24e69c6 100644 --- a/src/backend/catalog/partition.c +++ b/src/backend/catalog/partition.c @@ -36,6 +36,8 @@ #include "optimizer/clauses.h" #include "optimizer/planmain.h" #include "optimizer/var.h" +#include "parser/parse_func.h" +#include "parser/parse_oper.h" #include "rewrite/rewriteManip.h" #include "storage/lmgr.h" #include "utils/array.h" @@ -120,6 +122,7 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec); static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec); +static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec); static Oid get_partition_operator(PartitionKey key, int col, StrategyNumber strategy, bool *need_relabel); static List *generate_partition_qual(Relation rel); @@ -236,7 +239,8 @@ 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_LIST || + key->strategy == PARTITION_STRATEGY_HASH) { List *non_null_values = NIL;@@ -251,7 +255,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);- if (spec->strategy != PARTITION_STRATEGY_LIST) + if (spec->strategy != key->strategy) elog(ERROR, "invalid strategy in partition bound spec");foreach(c, spec->listdatums)
@@ -464,6 +468,7 @@ RelationBuildPartitionDesc(Relation rel)
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
{
boundinfo->has_null = found_null;
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
@@ -829,6 +834,18 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
break;
}+ case PARTITION_STRATEGY_HASH: + { + Assert(spec->strategy == PARTITION_STRATEGY_HASH); + + if (partdesc->nparts + 1 > key->partnparts) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("cannot create hash partition more than %d for %s", + key->partnparts, RelationGetRelationName(parent)))); + break; + } + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -916,6 +933,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound) my_qual = get_qual_for_range(key, spec); break;+ case PARTITION_STRATEGY_HASH: + Assert(spec->strategy == PARTITION_STRATEGY_LIST); + my_qual = get_qual_for_hash(key, spec); + break; + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -1146,6 +1168,84 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode, return pd; }+/* + * convert_expr_for_hash + * + * Converts a expr for a hash partition's constraint. + * expr is converted into 'abs(hashfunc(expr)) % npart". + * + * npart: number of partitions + * hashfunc: OID of hash function + */ +Expr * +convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc) +{ + FuncExpr *func, + *abs; + Expr *modexpr; + Oid modoid; + Oid int4oid[1] = {INT4OID}; + + ParseState *pstate = make_parsestate(NULL); + Value *val_npart = makeInteger(npart); + Node *const_npart = (Node *) make_const(pstate, val_npart, -1); + + /* hash function */ + func = makeFuncExpr(hashfunc, + INT4OID, + list_make1(expr), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* Abs */ + abs = makeFuncExpr(LookupFuncName(list_make1(makeString("abs")), 1, int4oid, false), + INT4OID, + list_make1(func), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* modulo by npart */ + modoid = LookupOperName(pstate, list_make1(makeString("%")), INT4OID, INT4OID, false, -1); + modexpr = make_opclause(modoid, INT4OID, false, (Expr*)abs, (Expr*)const_npart, 0, 0); + + return modexpr; +} + + +/* + * get_next_hash_partition_index + * + * Returns the minimal index which is not used for hash partition. + */ +int +get_next_hash_partition_index(Relation parent) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionDesc partdesc = RelationGetPartitionDesc(parent); + + int i; + bool *used = palloc0(sizeof(int) * key->partnparts); + + /* mark used for existing partition indexs */ + for (i = 0; i < partdesc->boundinfo->ndatums; i++) + { + Datum* datum = partdesc->boundinfo->datums[i]; + int idx = DatumGetInt16(datum[0]); + + if (!used[idx]) + used[idx] = true; + } + + /* find the minimal unused index */ + for (i = 0; i < key->partnparts; i++) + if (!used[i]) + break; + + return i; +} + /* Module-local functions *//*
@@ -1467,6 +1567,43 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
}/* + * get_qual_for_hash + * + * Returns a list of expressions to use as a hash partition's constraint. + */ +static List * +get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec) +{ + List *result; + Expr *keyCol; + Expr *expr; + Expr *opexpr; + Oid operoid; + ParseState *pstate = make_parsestate(NULL); + + /* Left operand */ + if (key->partattrs[0] != 0) + keyCol = (Expr *) makeVar(1, + key->partattrs[0], + key->parttypid[0], + key->parttypmod[0], + key->parttypcoll[0], + 0); + else + keyCol = (Expr *) copyObject(linitial(key->partexprs)); + + expr = convert_expr_for_hash(keyCol, key->partnparts, key->parthashfunc); + + /* equals the listdaums value */ + operoid = LookupOperName(pstate, list_make1(makeString("=")), INT4OID, INT4OID, false, -1); + opexpr = make_opclause(operoid, BOOLOID, false, expr, linitial(spec->listdatums), 0, 0); + + result = list_make1(opexpr); + + return result; +} + +/* * get_partition_operator * * Return oid of the operator of given strategy for a given partition key @@ -1730,6 +1867,11 @@ get_partition_for_tuple(PartitionDispatch *pd, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), errmsg("range partition key of row contains null"))); } + else if (key->strategy == PARTITION_STRATEGY_HASH) + { + values[0] = OidFunctionCall1(key->parthashfunc, values[0]); + values[0] = Int16GetDatum(Abs(DatumGetInt16(values[0])) % key->partnparts); + }if (partdesc->boundinfo->has_null && isnull[0]) /* Tuple maps to the null-accepting list partition */ @@ -1744,6 +1886,7 @@ get_partition_for_tuple(PartitionDispatch *pd, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: if (cur_offset >= 0 && equal) cur_index = partdesc->boundinfo->indexes[cur_offset]; else @@ -1968,6 +2111,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0], key->partcollation[0], bound_datums[0], diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index 3cea220..5a28cc0 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -41,6 +41,7 @@ #include "catalog/pg_inherits_fn.h" #include "catalog/pg_namespace.h" #include "catalog/pg_opclass.h" +#include "catalog/pg_proc.h" #include "catalog/pg_tablespace.h" #include "catalog/pg_trigger.h" #include "catalog/pg_type.h" @@ -77,6 +78,7 @@ #include "parser/parse_oper.h" #include "parser/parse_relation.h" #include "parser/parse_type.h" +#include "parser/parse_func.h" #include "parser/parse_utilcmd.h" #include "parser/parser.h" #include "pgstat.h" @@ -450,7 +452,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid, Oid oldrelid, void *arg); 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, +static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation); static void CreateInheritance(Relation child_rel, Relation parent_rel); static void RemoveInheritance(Relation child_rel, Relation parent_rel); @@ -799,8 +801,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, AttrNumber partattrs[PARTITION_MAX_KEYS]; Oid partopclass[PARTITION_MAX_KEYS]; Oid partcollation[PARTITION_MAX_KEYS]; + Oid partatttypes[PARTITION_MAX_KEYS]; List *partexprs = NIL; List *cmds = NIL; + Oid hashfuncOid = InvalidOid;/* * We need to transform the raw parsetrees corresponding to partition @@ -811,15 +815,40 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, stmt->partspec = transformPartitionSpec(rel, stmt->partspec, &strategy); ComputePartitionAttrs(rel, stmt->partspec->partParams, - partattrs, &partexprs, partopclass, + partattrs, partatttypes, &partexprs, partopclass, partcollation);partnatts = list_length(stmt->partspec->partParams); + + if (strategy == PARTITION_STRATEGY_HASH) + { + Oid funcrettype; + + if (partnatts != 1) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partition key must be 1 for hash partition"))); + + hashfuncOid = LookupFuncName(stmt->partspec->hashfunc, 1, partatttypes, false); + funcrettype = get_func_rettype(hashfuncOid); + if (funcrettype != INT4OID) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must return integer"))); + + if (func_volatile(hashfuncOid) != PROVOLATILE_IMMUTABLE) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must be marked IMMUTABLE"))); + + } + StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs, - partopclass, partcollation); + partopclass, partcollation, stmt->partspec->partnparts, hashfuncOid);- /* Force key columns to be NOT NULL when using range partitioning */ - if (strategy == PARTITION_STRATEGY_RANGE) + /* Force key columns to be NOT NULL when using range or hash partitioning */ + if (strategy == PARTITION_STRATEGY_RANGE || + strategy == PARTITION_STRATEGY_HASH) { for (i = 0; i < partnatts; i++) { @@ -12783,18 +12812,51 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) newspec->strategy = partspec->strategy; newspec->location = partspec->location; newspec->partParams = NIL; + newspec->partnparts = partspec->partnparts; + newspec->hashfunc = partspec->hashfunc;/* Parse partitioning strategy name */ if (!pg_strcasecmp(partspec->strategy, "list")) *strategy = PARTITION_STRATEGY_LIST; else if (!pg_strcasecmp(partspec->strategy, "range")) *strategy = PARTITION_STRATEGY_RANGE; + else if (!pg_strcasecmp(partspec->strategy, "hash")) + *strategy = PARTITION_STRATEGY_HASH; else ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("unrecognized partitioning strategy \"%s\"", partspec->strategy)));+ if (*strategy == PARTITION_STRATEGY_HASH) + { + if (partspec->partnparts < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be specified for hash partition"))); + else if (partspec->partnparts == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be greater than 0"))); + + if (list_length(partspec->hashfunc) == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function must be specified for hash partition"))); + } + else + { + if (partspec->partnparts >= 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions can be specified only for hash partition"))); + + if (list_length(partspec->hashfunc) > 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function can be specified only for hash partition"))); + } + /* * Create a dummy ParseState and insert the target relation as its sole * rangetable entry. We need a ParseState for transformExpr. @@ -12843,7 +12905,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) * Compute per-partition-column information from a list of PartitionElem's */ static void -ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, +ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation) { int attn; @@ -13010,6 +13072,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, "btree", BTREE_AM_OID);+ partatttypes[attn] = atttype; attn++; } } diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c index 05d8538..f4febc9 100644 --- a/src/backend/nodes/copyfuncs.c +++ b/src/backend/nodes/copyfuncs.c @@ -4232,6 +4232,8 @@ _copyPartitionSpec(const PartitionSpec *from)COPY_STRING_FIELD(strategy);
COPY_NODE_FIELD(partParams);
+ COPY_SCALAR_FIELD(partnparts);
+ COPY_NODE_FIELD(hashfunc);
COPY_LOCATION_FIELD(location);return newnode; diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c index d595cd7..d589eac 100644 --- a/src/backend/nodes/equalfuncs.c +++ b/src/backend/nodes/equalfuncs.c @@ -2725,6 +2725,8 @@ _equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b) { COMPARE_STRING_FIELD(strategy); COMPARE_NODE_FIELD(partParams); + COMPARE_SCALAR_FIELD(partnparts); + COMPARE_NODE_FIELD(hashfunc); COMPARE_LOCATION_FIELD(location);return true; diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index b3802b4..d6db80e 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -3318,6 +3318,8 @@ _outPartitionSpec(StringInfo str, const PartitionSpec *node)WRITE_STRING_FIELD(strategy);
WRITE_NODE_FIELD(partParams);
+ WRITE_INT_FIELD(partnparts);
+ WRITE_NODE_FIELD(hashfunc);
WRITE_LOCATION_FIELD(location);
}diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index e833b2e..b67140d 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -574,6 +574,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); %type <list> partbound_datum_list %type <partrange_datum> PartitionRangeDatum %type <list> range_datum_list +%type <ival> hash_partitions +%type <list> hash_function/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -627,7 +629,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING
- HANDLER HAVING HEADER_P HOLD HOUR_P + HANDLER HASH HAVING HEADER_P HOLD HOUR_PIDENTITY_P IF_P ILIKE IMMEDIATE IMMUTABLE IMPLICIT_P IMPORT_P IN_P
INCLUDING INCREMENT INDEX INDEXES INHERIT INHERITS INITIALLY INLINE_P
@@ -651,7 +653,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
OBJECT_P OF OFF OFFSET OIDS OLD ON ONLY OPERATOR OPTION OPTIONS OR
ORDER ORDINALITY OUT_P OUTER_P OVER OVERLAPS OVERLAY OWNED OWNER- PARALLEL PARSER PARTIAL PARTITION PASSING PASSWORD PLACING PLANS POLICY + PARALLEL PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PLACING PLANS POLICY POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROGRAM PUBLICATION@@ -2587,6 +2589,16 @@ ForValues:
$$ = (Node *) n; } + + /* a HASH partition */ + | /*EMPTY*/ + { + PartitionBoundSpec *n = makeNode(PartitionBoundSpec); + + n->strategy = PARTITION_STRATEGY_HASH; + + $$ = (Node *) n; + } ;partbound_datum:
@@ -3666,7 +3678,7 @@ OptPartitionSpec: PartitionSpec { $$ = $1; }
| /*EMPTY*/ { $$ = NULL; }
;-PartitionSpec: PARTITION BY part_strategy '(' part_params ')' +PartitionSpec: PARTITION BY part_strategy '(' part_params ')' hash_partitions hash_function { PartitionSpec *n = makeNode(PartitionSpec);@@ -3674,10 +3686,21 @@ PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
n->partParams = $5;
n->location = @1;+ n->partnparts = $7; + n->hashfunc = $8; + $$ = n; } ;+hash_partitions: PARTITIONS Iconst { $$ = $2; } + | /*EMPTY*/ { $$ = -1; } + ; + +hash_function: USING handler_name { $$ = $2; } + | /*EMPTY*/ { $$ = NULL; } + ; + part_strategy: IDENT { $$ = $1; } | unreserved_keyword { $$ = pstrdup($1); } ; @@ -14377,6 +14400,7 @@ unreserved_keyword: | GLOBAL | GRANTED | HANDLER + | HASH | HEADER_P | HOLD | HOUR_P @@ -14448,6 +14472,7 @@ unreserved_keyword: | PARSER | PARTIAL | PARTITION + | PARTITIONS | PASSING | PASSWORD | PLANS diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c index ff2bab6..8e1be31 100644 --- a/src/backend/parser/parse_utilcmd.c +++ b/src/backend/parser/parse_utilcmd.c @@ -40,6 +40,7 @@ #include "catalog/pg_opclass.h" #include "catalog/pg_operator.h" #include "catalog/pg_type.h" +#include "catalog/partition.h" #include "commands/comment.h" #include "commands/defrem.h" #include "commands/tablecmds.h" @@ -3252,6 +3253,24 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound) ++i; } } + else if (strategy == PARTITION_STRATEGY_HASH) + { + Value *conval; + Node *value; + int index; + + if (spec->strategy != PARTITION_STRATEGY_HASH) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("invalid bound specification for a hash partition"))); + + index = get_next_hash_partition_index(parent); + + /* store the partition index as a listdatums value */ + conval = makeInteger(index); + value = (Node *) make_const(pstate, conval, -1); + result_spec->listdatums = list_make1(value); + } else elog(ERROR, "unexpected partition strategy: %d", (int) strategy);diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c index b27b77d..fab6eea 100644 --- a/src/backend/utils/adt/ruleutils.c +++ b/src/backend/utils/adt/ruleutils.c @@ -1423,7 +1423,7 @@ pg_get_indexdef_worker(Oid indexrelid, int colno, * * 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) @@ -1513,6 +1513,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags) case PARTITION_STRATEGY_RANGE: appendStringInfo(&buf, "RANGE"); break; + case PARTITION_STRATEGY_HASH: + appendStringInfo(&buf, "HASH"); + break; default: elog(ERROR, "unexpected partition strategy: %d", (int) form->partstrat); @@ -8520,6 +8523,9 @@ get_rule_expr(Node *node, deparse_context *context, appendStringInfoString(buf, ")"); break;+ case PARTITION_STRATEGY_HASH: + break; + default: elog(ERROR, "unrecognized partition strategy: %d", (int) spec->strategy); diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c index 9001e20..829e4d2 100644 --- a/src/backend/utils/cache/relcache.c +++ b/src/backend/utils/cache/relcache.c @@ -855,6 +855,9 @@ RelationBuildPartitionKey(Relation relation) key->strategy = form->partstrat; key->partnatts = form->partnatts;+ key->partnparts = form->partnparts; + key->parthashfunc = form->parthashfunc; + /* * We can rely on the first variable-length attribute being mapped to the * relevant field of the catalog's C struct, because all previous @@ -999,6 +1002,9 @@ copy_partition_key(PartitionKey fromkey) newkey->strategy = fromkey->strategy; newkey->partnatts = n = fromkey->partnatts;+ newkey->partnparts = fromkey->partnparts; + newkey->parthashfunc = fromkey->parthashfunc; + newkey->partattrs = (AttrNumber *) palloc(n * sizeof(AttrNumber)); memcpy(newkey->partattrs, fromkey->partattrs, n * sizeof(AttrNumber));diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h index 1187797..367e2f8 100644 --- a/src/include/catalog/heap.h +++ b/src/include/catalog/heap.h @@ -141,7 +141,7 @@ extern void StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation); + Oid *partcollation, int16 partnparts, Oid hashfunc); extern void RemovePartitionKeyByRelId(Oid relid); extern void StorePartitionBound(Relation rel, Relation parent, Node *bound);diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h index b195d1a..80f4b0e 100644 --- a/src/include/catalog/partition.h +++ b/src/include/catalog/partition.h @@ -89,4 +89,6 @@ extern int get_partition_for_tuple(PartitionDispatch *pd, TupleTableSlot *slot, EState *estate, Oid *failed_at); +extern Expr *convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc); +extern int get_next_hash_partition_index(Relation parent); #endif /* PARTITION_H */ diff --git a/src/include/catalog/pg_partitioned_table.h b/src/include/catalog/pg_partitioned_table.h index bdff36a..69e509c 100644 --- a/src/include/catalog/pg_partitioned_table.h +++ b/src/include/catalog/pg_partitioned_table.h @@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS char partstrat; /* partitioning strategy */ int16 partnatts; /* number of partition key columns */+ int16 partnparts;
+ Oid parthashfunc;
+
/*
* variable-length fields start here, but we allow direct access to
* partattrs via the C struct. That's because the first variable-length
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;/* ---------------- @@ -62,13 +67,15 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table; * compiler constants for pg_partitioned_table * ---------------- */ -#define Natts_pg_partitioned_table 7 +#define Natts_pg_partitioned_table 9 #define Anum_pg_partitioned_table_partrelid 1 #define Anum_pg_partitioned_table_partstrat 2 #define Anum_pg_partitioned_table_partnatts 3 -#define Anum_pg_partitioned_table_partattrs 4 -#define Anum_pg_partitioned_table_partclass 5 -#define Anum_pg_partitioned_table_partcollation 6 -#define Anum_pg_partitioned_table_partexprs 7 +#define Anum_pg_partitioned_table_partnparts 4 +#define Anum_pg_partitioned_table_parthashfunc 5 +#define Anum_pg_partitioned_table_partattrs 6 +#define Anum_pg_partitioned_table_partclass 7 +#define Anum_pg_partitioned_table_partcollation 8 +#define Anum_pg_partitioned_table_partexprs 9#endif /* PG_PARTITIONED_TABLE_H */ diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index 5afc3eb..1c3474f 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -730,11 +730,14 @@ typedef struct PartitionSpec NodeTag type; char *strategy; /* partitioning strategy ('list' or 'range') */ List *partParams; /* List of PartitionElems */ + int partnparts; + List *hashfunc; int location; /* token location, or -1 if unknown */ } PartitionSpec;#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'/* * PartitionBoundSpec - a partition bound specification diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index 985d650..0597939 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -180,6 +180,7 @@ PG_KEYWORD("greatest", GREATEST, COL_NAME_KEYWORD) PG_KEYWORD("group", GROUP_P, RESERVED_KEYWORD) PG_KEYWORD("grouping", GROUPING, COL_NAME_KEYWORD) PG_KEYWORD("handler", HANDLER, UNRESERVED_KEYWORD) +PG_KEYWORD("hash", HASH, UNRESERVED_KEYWORD) PG_KEYWORD("having", HAVING, RESERVED_KEYWORD) PG_KEYWORD("header", HEADER_P, UNRESERVED_KEYWORD) PG_KEYWORD("hold", HOLD, UNRESERVED_KEYWORD) @@ -291,6 +292,7 @@ PG_KEYWORD("parallel", PARALLEL, UNRESERVED_KEYWORD) PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD) PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD) PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD) +PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD) PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD) PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD) PG_KEYWORD("placing", PLACING, RESERVED_KEYWORD) diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h index a617a7c..660adfb 100644 --- a/src/include/utils/rel.h +++ b/src/include/utils/rel.h @@ -62,6 +62,9 @@ typedef struct PartitionKeyData Oid *partopcintype; /* OIDs of opclass declared input data types */ FmgrInfo *partsupfunc; /* lookup info for support funcs */+ int16 partnparts; /* number of hash partitions */ + Oid parthashfunc; /* OID of hash function */ + /* Partitioning collation per attribute */ Oid *partcollation;--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers--
Best regards,
Aleksander Alekseev--
Yugo Nagata <nagata@sraoss.co.jp>
--
Best regards,
Aleksander Alekseev
On 01.03.2017 05:14, Amit Langote wrote:
Nagata-san,
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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hInstead of having to create each partition individually, wouldn't it be
better if the following commandCREATE TABLE h (i int) PARTITION BY HASH (i) PARTITIONS 3;
created the partitions *automatically*?
It's a good idea but in this case we can't create hash-partition that is
also partitioned table, and as a consequence we are unable to create
subpartitions. My understanding is that the table can be partitioned
only using CREATE TABLE statement, not ALTER TABLE. For this reason the
new created partitions are only regular tables.
We can achieve desired result through creating a separate partitioned
table and making the DETACH/ATTACH manipulation, though. But IMO it's
not flexible case.
It would be a good thing if a regular table could be partitioned through
separate command. Then your idea would not be restrictive.
--
Maksim Milyutin
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
We can achieve desired result through creating a separate partitioned table
and making the DETACH/ATTACH manipulation, though. But IMO it's not flexible
case.
I think it would be great to allow end user to decide. If user is
not interested in subpartitions he or she can use syntax like 'CREATE
TABLE ... PARTITION BY HAHS(i) PARTITIONS 3 CREATE AUTOMATICALLY;' or
maybe a build-in procedure for this. Otherwise there is also
ATTACH/DETACH syntax available.
Anyway all of this is something that could be discussed infinitely and
not necessarily should be included in this concrete patch. We could
probably agree that 3 or 4 separately discussed, reviewed and tested
patches are better than one huge patch that will be moved to the next
commitfest because of disagreements regarding a syntax.
On Wed, Mar 01, 2017 at 05:10:34PM +0300, Maksim Milyutin wrote:
On 01.03.2017 05:14, Amit Langote wrote:
Nagata-san,
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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hInstead of having to create each partition individually, wouldn't it be
better if the following commandCREATE TABLE h (i int) PARTITION BY HASH (i) PARTITIONS 3;
created the partitions *automatically*?
It's a good idea but in this case we can't create hash-partition that is
also partitioned table, and as a consequence we are unable to create
subpartitions. My understanding is that the table can be partitioned only
using CREATE TABLE statement, not ALTER TABLE. For this reason the new
created partitions are only regular tables.We can achieve desired result through creating a separate partitioned table
and making the DETACH/ATTACH manipulation, though. But IMO it's not flexible
case.It would be a good thing if a regular table could be partitioned through
separate command. Then your idea would not be restrictive.--
Maksim Milyutin
Postgres Professional: http://www.postgrespro.com
Russian Postgres Company--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
--
Best regards,
Aleksander Alekseev
Hi Aleksander ,
Thank you for reviewing the patch.
On Wed, 1 Mar 2017 17:08:49 +0300
Aleksander Alekseev <a.alekseev@postgrespro.ru> wrote:
Hi, Yugo.
Today I've had an opportunity to take a closer look on this patch. Here are
a few things that bother me.1a) There are missing commends here:
``` --- a/src/include/catalog/pg_partitioned_table.h +++ b/src/include/catalog/pg_partitioned_table.h @@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS char partstrat; /* partitioning strategy */ int16 partnatts; /* number of partition key columns */+ int16 partnparts; + Oid parthashfunc; + ```1b) ... and here:
``` --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -730,11 +730,14 @@ typedef struct PartitionSpec NodeTag type; char *strategy; /* partitioning strategy ('list' or 'range') */ List *partParams; /* List of PartitionElems */ + int partnparts; + List *hashfunc; int location; /* token location, or -1 if unknown */ } PartitionSpec; ```
ok, I'll add comments for these members;
2) I believe new empty lines in patches are generally not welcomed by
community:```
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;
```
I'll remove it from the patch.
3) One test fails on my laptop (Arch Linux, x64) [1]:
``` *************** *** 344,350 **** 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 --- 344,350 ---- CREATE TABLE partitioned ( a int ) PARTITION BY HASH (a); ! ERROR: number of partitions must be specified for hash partition -- specified column must be present in the table CREATE TABLE partitioned ( a int ```
These are expected behaviors in the current patch. However, there
are some discussions on the specification about CREATE TABLE, so
it may be changed.
Exact script I'm using for building and testing PostgreSQL could be
found here [2].4) As I already mentioned - missing documentation.
I think writing the documentation should be waited fo the specification
getting a consensus.
In general patch looks quite good to me. I personally believe it has all
the changes to be accepted in current commitfest. Naturally if community
will come to a consensus regarding keywords, whether all partitions
should be created automatically, etc :)[1] http://afiskon.ru/s/dd/20cbe21934_regression.diffs.txt
[2] http://afiskon.ru/s/76/a4fb71739c_full-build.sh.txtOn Wed, Mar 01, 2017 at 06:10:10PM +0900, Yugo Nagata wrote:
Hi Aleksander,
On Tue, 28 Feb 2017 18:05:36 +0300
Aleksander Alekseev <a.alekseev@postgrespro.ru> wrote:Hi, Yugo.
Looks like a great feature! I'm going to take a closer look on your code
and write a feedback shortly. For now I can only tell that you forgot
to include some documentation in the patch.Thank you for looking into it. I'm forward to your feedback.
This is a proof of concept patch and additional documentation
is not included. I'll add this after reaching a consensus
on the specification of the feature.I've added a corresponding entry to current commitfest [1]. Hope you
don't mind. If it's not too much trouble could you please register on a
commitfest site and add yourself to this entry as an author? I'm pretty
sure someone is using this information for writing release notes or
something like this.Thank you for registering it to the commitfest. I have added me as an auther.
[1] https://commitfest.postgresql.org/13/1059/
On Tue, Feb 28, 2017 at 11:33:13PM +0900, Yugo Nagata wrote:
Hi all,
Now we have a declarative partitioning, but hash partitioning is not
implemented yet. Attached is a POC patch to add the hash partitioning
feature. I know we will need more discussions about the syntax and other
specifications before going ahead the project, but I think this runnable
code might help to discuss what and how we implement this.* Description
In this patch, the hash partitioning implementation is basically based
on the list partitioning mechanism. However, partition bounds cannot be
specified explicitly, but this is used internally as hash partition
index, which is calculated when a partition is created or attached.The tentative syntax to create a partitioned table is as bellow;
CREATE TABLE h (i int) PARTITION BY HASH(i) PARTITIONS 3 USING hashint4;
The number of partitions is specified by PARTITIONS, which is currently
constant and cannot be changed, but I think this is needed to be changed in
some manner. A hash function is specified by USING. Maybe, specifying hash
function may be ommitted, and in this case, a default hash function
corresponding to key type will be used.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.When trying create partitions more than the number specified
by PARTITIONS, it gets an error.postgres=# create table h4 partition of h;
ERROR: cannot create hash partition more than 3 for hAn inserted record is stored in a partition whose index equals
abs(hashfunc(key)) % <number_of_partitions>. In the above
example, this is abs(hashint4(i))%3.postgres=# insert into h (select generate_series(0,20));
INSERT 0 21postgres=# select *,tableoid::regclass from h;
i | tableoid
----+----------
0 | h1
1 | h1
2 | h1
4 | h1
8 | h1
10 | h1
11 | h1
14 | h1
15 | h1
17 | h1
20 | h1
5 | h2
12 | h2
13 | h2
16 | h2
19 | h2
3 | h3
6 | h3
7 | h3
9 | h3
18 | h3
(21 rows)* Todo / discussions
In this patch, we cannot change the number of partitions specified
by PARTITIONS. I we can change this, the partitioning rule
(<partition index> = abs(hashfunc(key)) % <number_of_partitions>)
is also changed and then we need reallocatiing records between
partitions.In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.Currently, even when we issue SELECT query with a condition,
postgres looks into all partitions regardless of each partition's
constraint, because this is complicated such like "abs(hashint4(i))%3 = 0".postgres=# explain select * from h where i = 10;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..125.62 rows=40 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: (i = 10)
-> Seq Scan on h1 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h2 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
-> Seq Scan on h3 (cost=0.00..41.88 rows=13 width=4)
Filter: (i = 10)
(9 rows)However, if we modify a condition into a same expression
as the partitions constraint, postgres can exclude unrelated
table from search targets. So, we might avoid the problem
by converting the qual properly before calling predicate_refuted_by().postgres=# explain select * from h where abs(hashint4(i))%3 = abs(hashint4(10))%3;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..61.00 rows=14 width=4)
-> Seq Scan on h (cost=0.00..0.00 rows=1 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
-> Seq Scan on h3 (cost=0.00..61.00 rows=13 width=4)
Filter: ((abs(hashint4(i)) % 3) = 2)
(5 rows)Best regards,
Yugo Nagata--
Yugo Nagata <nagata@sraoss.co.jp>diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c index 41c0056..3820920 100644 --- a/src/backend/catalog/heap.c +++ b/src/backend/catalog/heap.c @@ -3074,7 +3074,7 @@ StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation) + Oid *partcollation, int16 partnparts, Oid hashfunc) { int i; int2vector *partattrs_vec; @@ -3121,6 +3121,8 @@ StorePartitionKey(Relation rel, values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel)); values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy); values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts); + values[Anum_pg_partitioned_table_partnparts - 1] = Int16GetDatum(partnparts); + values[Anum_pg_partitioned_table_parthashfunc - 1] = ObjectIdGetDatum(hashfunc); values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec); values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec); values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec); diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c index 4bcef58..24e69c6 100644 --- a/src/backend/catalog/partition.c +++ b/src/backend/catalog/partition.c @@ -36,6 +36,8 @@ #include "optimizer/clauses.h" #include "optimizer/planmain.h" #include "optimizer/var.h" +#include "parser/parse_func.h" +#include "parser/parse_oper.h" #include "rewrite/rewriteManip.h" #include "storage/lmgr.h" #include "utils/array.h" @@ -120,6 +122,7 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec); static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec); +static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec); static Oid get_partition_operator(PartitionKey key, int col, StrategyNumber strategy, bool *need_relabel); static List *generate_partition_qual(Relation rel); @@ -236,7 +239,8 @@ 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_LIST || + key->strategy == PARTITION_STRATEGY_HASH) { List *non_null_values = NIL;@@ -251,7 +255,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);- if (spec->strategy != PARTITION_STRATEGY_LIST) + if (spec->strategy != key->strategy) elog(ERROR, "invalid strategy in partition bound spec");foreach(c, spec->listdatums)
@@ -464,6 +468,7 @@ RelationBuildPartitionDesc(Relation rel)
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
{
boundinfo->has_null = found_null;
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
@@ -829,6 +834,18 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
break;
}+ case PARTITION_STRATEGY_HASH: + { + Assert(spec->strategy == PARTITION_STRATEGY_HASH); + + if (partdesc->nparts + 1 > key->partnparts) + ereport(ERROR, + (errcode(ERRCODE_INVALID_OBJECT_DEFINITION), + errmsg("cannot create hash partition more than %d for %s", + key->partnparts, RelationGetRelationName(parent)))); + break; + } + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -916,6 +933,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound) my_qual = get_qual_for_range(key, spec); break;+ case PARTITION_STRATEGY_HASH: + Assert(spec->strategy == PARTITION_STRATEGY_LIST); + my_qual = get_qual_for_hash(key, spec); + break; + default: elog(ERROR, "unexpected partition strategy: %d", (int) key->strategy); @@ -1146,6 +1168,84 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode, return pd; }+/* + * convert_expr_for_hash + * + * Converts a expr for a hash partition's constraint. + * expr is converted into 'abs(hashfunc(expr)) % npart". + * + * npart: number of partitions + * hashfunc: OID of hash function + */ +Expr * +convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc) +{ + FuncExpr *func, + *abs; + Expr *modexpr; + Oid modoid; + Oid int4oid[1] = {INT4OID}; + + ParseState *pstate = make_parsestate(NULL); + Value *val_npart = makeInteger(npart); + Node *const_npart = (Node *) make_const(pstate, val_npart, -1); + + /* hash function */ + func = makeFuncExpr(hashfunc, + INT4OID, + list_make1(expr), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* Abs */ + abs = makeFuncExpr(LookupFuncName(list_make1(makeString("abs")), 1, int4oid, false), + INT4OID, + list_make1(func), + 0, + 0, + COERCE_EXPLICIT_CALL); + + /* modulo by npart */ + modoid = LookupOperName(pstate, list_make1(makeString("%")), INT4OID, INT4OID, false, -1); + modexpr = make_opclause(modoid, INT4OID, false, (Expr*)abs, (Expr*)const_npart, 0, 0); + + return modexpr; +} + + +/* + * get_next_hash_partition_index + * + * Returns the minimal index which is not used for hash partition. + */ +int +get_next_hash_partition_index(Relation parent) +{ + PartitionKey key = RelationGetPartitionKey(parent); + PartitionDesc partdesc = RelationGetPartitionDesc(parent); + + int i; + bool *used = palloc0(sizeof(int) * key->partnparts); + + /* mark used for existing partition indexs */ + for (i = 0; i < partdesc->boundinfo->ndatums; i++) + { + Datum* datum = partdesc->boundinfo->datums[i]; + int idx = DatumGetInt16(datum[0]); + + if (!used[idx]) + used[idx] = true; + } + + /* find the minimal unused index */ + for (i = 0; i < key->partnparts; i++) + if (!used[i]) + break; + + return i; +} + /* Module-local functions *//*
@@ -1467,6 +1567,43 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
}/* + * get_qual_for_hash + * + * Returns a list of expressions to use as a hash partition's constraint. + */ +static List * +get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec) +{ + List *result; + Expr *keyCol; + Expr *expr; + Expr *opexpr; + Oid operoid; + ParseState *pstate = make_parsestate(NULL); + + /* Left operand */ + if (key->partattrs[0] != 0) + keyCol = (Expr *) makeVar(1, + key->partattrs[0], + key->parttypid[0], + key->parttypmod[0], + key->parttypcoll[0], + 0); + else + keyCol = (Expr *) copyObject(linitial(key->partexprs)); + + expr = convert_expr_for_hash(keyCol, key->partnparts, key->parthashfunc); + + /* equals the listdaums value */ + operoid = LookupOperName(pstate, list_make1(makeString("=")), INT4OID, INT4OID, false, -1); + opexpr = make_opclause(operoid, BOOLOID, false, expr, linitial(spec->listdatums), 0, 0); + + result = list_make1(opexpr); + + return result; +} + +/* * get_partition_operator * * Return oid of the operator of given strategy for a given partition key @@ -1730,6 +1867,11 @@ get_partition_for_tuple(PartitionDispatch *pd, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), errmsg("range partition key of row contains null"))); } + else if (key->strategy == PARTITION_STRATEGY_HASH) + { + values[0] = OidFunctionCall1(key->parthashfunc, values[0]); + values[0] = Int16GetDatum(Abs(DatumGetInt16(values[0])) % key->partnparts); + }if (partdesc->boundinfo->has_null && isnull[0]) /* Tuple maps to the null-accepting list partition */ @@ -1744,6 +1886,7 @@ get_partition_for_tuple(PartitionDispatch *pd, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: if (cur_offset >= 0 && equal) cur_index = partdesc->boundinfo->indexes[cur_offset]; else @@ -1968,6 +2111,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo, switch (key->strategy) { case PARTITION_STRATEGY_LIST: + case PARTITION_STRATEGY_HASH: cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0], key->partcollation[0], bound_datums[0], diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index 3cea220..5a28cc0 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -41,6 +41,7 @@ #include "catalog/pg_inherits_fn.h" #include "catalog/pg_namespace.h" #include "catalog/pg_opclass.h" +#include "catalog/pg_proc.h" #include "catalog/pg_tablespace.h" #include "catalog/pg_trigger.h" #include "catalog/pg_type.h" @@ -77,6 +78,7 @@ #include "parser/parse_oper.h" #include "parser/parse_relation.h" #include "parser/parse_type.h" +#include "parser/parse_func.h" #include "parser/parse_utilcmd.h" #include "parser/parser.h" #include "pgstat.h" @@ -450,7 +452,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid, Oid oldrelid, void *arg); 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, +static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation); static void CreateInheritance(Relation child_rel, Relation parent_rel); static void RemoveInheritance(Relation child_rel, Relation parent_rel); @@ -799,8 +801,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, AttrNumber partattrs[PARTITION_MAX_KEYS]; Oid partopclass[PARTITION_MAX_KEYS]; Oid partcollation[PARTITION_MAX_KEYS]; + Oid partatttypes[PARTITION_MAX_KEYS]; List *partexprs = NIL; List *cmds = NIL; + Oid hashfuncOid = InvalidOid;/* * We need to transform the raw parsetrees corresponding to partition @@ -811,15 +815,40 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId, stmt->partspec = transformPartitionSpec(rel, stmt->partspec, &strategy); ComputePartitionAttrs(rel, stmt->partspec->partParams, - partattrs, &partexprs, partopclass, + partattrs, partatttypes, &partexprs, partopclass, partcollation);partnatts = list_length(stmt->partspec->partParams); + + if (strategy == PARTITION_STRATEGY_HASH) + { + Oid funcrettype; + + if (partnatts != 1) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partition key must be 1 for hash partition"))); + + hashfuncOid = LookupFuncName(stmt->partspec->hashfunc, 1, partatttypes, false); + funcrettype = get_func_rettype(hashfuncOid); + if (funcrettype != INT4OID) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must return integer"))); + + if (func_volatile(hashfuncOid) != PROVOLATILE_IMMUTABLE) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function for partitioning must be marked IMMUTABLE"))); + + } + StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs, - partopclass, partcollation); + partopclass, partcollation, stmt->partspec->partnparts, hashfuncOid);- /* Force key columns to be NOT NULL when using range partitioning */ - if (strategy == PARTITION_STRATEGY_RANGE) + /* Force key columns to be NOT NULL when using range or hash partitioning */ + if (strategy == PARTITION_STRATEGY_RANGE || + strategy == PARTITION_STRATEGY_HASH) { for (i = 0; i < partnatts; i++) { @@ -12783,18 +12812,51 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) newspec->strategy = partspec->strategy; newspec->location = partspec->location; newspec->partParams = NIL; + newspec->partnparts = partspec->partnparts; + newspec->hashfunc = partspec->hashfunc;/* Parse partitioning strategy name */ if (!pg_strcasecmp(partspec->strategy, "list")) *strategy = PARTITION_STRATEGY_LIST; else if (!pg_strcasecmp(partspec->strategy, "range")) *strategy = PARTITION_STRATEGY_RANGE; + else if (!pg_strcasecmp(partspec->strategy, "hash")) + *strategy = PARTITION_STRATEGY_HASH; else ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("unrecognized partitioning strategy \"%s\"", partspec->strategy)));+ if (*strategy == PARTITION_STRATEGY_HASH) + { + if (partspec->partnparts < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be specified for hash partition"))); + else if (partspec->partnparts == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions must be greater than 0"))); + + if (list_length(partspec->hashfunc) == 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function must be specified for hash partition"))); + } + else + { + if (partspec->partnparts >= 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("number of partitions can be specified only for hash partition"))); + + if (list_length(partspec->hashfunc) > 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("hash function can be specified only for hash partition"))); + } + /* * Create a dummy ParseState and insert the target relation as its sole * rangetable entry. We need a ParseState for transformExpr. @@ -12843,7 +12905,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy) * Compute per-partition-column information from a list of PartitionElem's */ static void -ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, +ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes, List **partexprs, Oid *partopclass, Oid *partcollation) { int attn; @@ -13010,6 +13072,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, "btree", BTREE_AM_OID);+ partatttypes[attn] = atttype; attn++; } } diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c index 05d8538..f4febc9 100644 --- a/src/backend/nodes/copyfuncs.c +++ b/src/backend/nodes/copyfuncs.c @@ -4232,6 +4232,8 @@ _copyPartitionSpec(const PartitionSpec *from)COPY_STRING_FIELD(strategy);
COPY_NODE_FIELD(partParams);
+ COPY_SCALAR_FIELD(partnparts);
+ COPY_NODE_FIELD(hashfunc);
COPY_LOCATION_FIELD(location);return newnode; diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c index d595cd7..d589eac 100644 --- a/src/backend/nodes/equalfuncs.c +++ b/src/backend/nodes/equalfuncs.c @@ -2725,6 +2725,8 @@ _equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b) { COMPARE_STRING_FIELD(strategy); COMPARE_NODE_FIELD(partParams); + COMPARE_SCALAR_FIELD(partnparts); + COMPARE_NODE_FIELD(hashfunc); COMPARE_LOCATION_FIELD(location);return true; diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index b3802b4..d6db80e 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -3318,6 +3318,8 @@ _outPartitionSpec(StringInfo str, const PartitionSpec *node)WRITE_STRING_FIELD(strategy);
WRITE_NODE_FIELD(partParams);
+ WRITE_INT_FIELD(partnparts);
+ WRITE_NODE_FIELD(hashfunc);
WRITE_LOCATION_FIELD(location);
}diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index e833b2e..b67140d 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -574,6 +574,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query); %type <list> partbound_datum_list %type <partrange_datum> PartitionRangeDatum %type <list> range_datum_list +%type <ival> hash_partitions +%type <list> hash_function/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -627,7 +629,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);GLOBAL GRANT GRANTED GREATEST GROUP_P GROUPING
- HANDLER HAVING HEADER_P HOLD HOUR_P + HANDLER HASH HAVING HEADER_P HOLD HOUR_PIDENTITY_P IF_P ILIKE IMMEDIATE IMMUTABLE IMPLICIT_P IMPORT_P IN_P
INCLUDING INCREMENT INDEX INDEXES INHERIT INHERITS INITIALLY INLINE_P
@@ -651,7 +653,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
OBJECT_P OF OFF OFFSET OIDS OLD ON ONLY OPERATOR OPTION OPTIONS OR
ORDER ORDINALITY OUT_P OUTER_P OVER OVERLAPS OVERLAY OWNED OWNER- PARALLEL PARSER PARTIAL PARTITION PASSING PASSWORD PLACING PLANS POLICY + PARALLEL PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PLACING PLANS POLICY POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROGRAM PUBLICATION@@ -2587,6 +2589,16 @@ ForValues:
$$ = (Node *) n; } + + /* a HASH partition */ + | /*EMPTY*/ + { + PartitionBoundSpec *n = makeNode(PartitionBoundSpec); + + n->strategy = PARTITION_STRATEGY_HASH; + + $$ = (Node *) n; + } ;partbound_datum:
@@ -3666,7 +3678,7 @@ OptPartitionSpec: PartitionSpec { $$ = $1; }
| /*EMPTY*/ { $$ = NULL; }
;-PartitionSpec: PARTITION BY part_strategy '(' part_params ')' +PartitionSpec: PARTITION BY part_strategy '(' part_params ')' hash_partitions hash_function { PartitionSpec *n = makeNode(PartitionSpec);@@ -3674,10 +3686,21 @@ PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
n->partParams = $5;
n->location = @1;+ n->partnparts = $7; + n->hashfunc = $8; + $$ = n; } ;+hash_partitions: PARTITIONS Iconst { $$ = $2; } + | /*EMPTY*/ { $$ = -1; } + ; + +hash_function: USING handler_name { $$ = $2; } + | /*EMPTY*/ { $$ = NULL; } + ; + part_strategy: IDENT { $$ = $1; } | unreserved_keyword { $$ = pstrdup($1); } ; @@ -14377,6 +14400,7 @@ unreserved_keyword: | GLOBAL | GRANTED | HANDLER + | HASH | HEADER_P | HOLD | HOUR_P @@ -14448,6 +14472,7 @@ unreserved_keyword: | PARSER | PARTIAL | PARTITION + | PARTITIONS | PASSING | PASSWORD | PLANS diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c index ff2bab6..8e1be31 100644 --- a/src/backend/parser/parse_utilcmd.c +++ b/src/backend/parser/parse_utilcmd.c @@ -40,6 +40,7 @@ #include "catalog/pg_opclass.h" #include "catalog/pg_operator.h" #include "catalog/pg_type.h" +#include "catalog/partition.h" #include "commands/comment.h" #include "commands/defrem.h" #include "commands/tablecmds.h" @@ -3252,6 +3253,24 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound) ++i; } } + else if (strategy == PARTITION_STRATEGY_HASH) + { + Value *conval; + Node *value; + int index; + + if (spec->strategy != PARTITION_STRATEGY_HASH) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("invalid bound specification for a hash partition"))); + + index = get_next_hash_partition_index(parent); + + /* store the partition index as a listdatums value */ + conval = makeInteger(index); + value = (Node *) make_const(pstate, conval, -1); + result_spec->listdatums = list_make1(value); + } else elog(ERROR, "unexpected partition strategy: %d", (int) strategy);diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c index b27b77d..fab6eea 100644 --- a/src/backend/utils/adt/ruleutils.c +++ b/src/backend/utils/adt/ruleutils.c @@ -1423,7 +1423,7 @@ pg_get_indexdef_worker(Oid indexrelid, int colno, * * 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) @@ -1513,6 +1513,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags) case PARTITION_STRATEGY_RANGE: appendStringInfo(&buf, "RANGE"); break; + case PARTITION_STRATEGY_HASH: + appendStringInfo(&buf, "HASH"); + break; default: elog(ERROR, "unexpected partition strategy: %d", (int) form->partstrat); @@ -8520,6 +8523,9 @@ get_rule_expr(Node *node, deparse_context *context, appendStringInfoString(buf, ")"); break;+ case PARTITION_STRATEGY_HASH: + break; + default: elog(ERROR, "unrecognized partition strategy: %d", (int) spec->strategy); diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c index 9001e20..829e4d2 100644 --- a/src/backend/utils/cache/relcache.c +++ b/src/backend/utils/cache/relcache.c @@ -855,6 +855,9 @@ RelationBuildPartitionKey(Relation relation) key->strategy = form->partstrat; key->partnatts = form->partnatts;+ key->partnparts = form->partnparts; + key->parthashfunc = form->parthashfunc; + /* * We can rely on the first variable-length attribute being mapped to the * relevant field of the catalog's C struct, because all previous @@ -999,6 +1002,9 @@ copy_partition_key(PartitionKey fromkey) newkey->strategy = fromkey->strategy; newkey->partnatts = n = fromkey->partnatts;+ newkey->partnparts = fromkey->partnparts; + newkey->parthashfunc = fromkey->parthashfunc; + newkey->partattrs = (AttrNumber *) palloc(n * sizeof(AttrNumber)); memcpy(newkey->partattrs, fromkey->partattrs, n * sizeof(AttrNumber));diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h index 1187797..367e2f8 100644 --- a/src/include/catalog/heap.h +++ b/src/include/catalog/heap.h @@ -141,7 +141,7 @@ extern void StorePartitionKey(Relation rel, AttrNumber *partattrs, List *partexprs, Oid *partopclass, - Oid *partcollation); + Oid *partcollation, int16 partnparts, Oid hashfunc); extern void RemovePartitionKeyByRelId(Oid relid); extern void StorePartitionBound(Relation rel, Relation parent, Node *bound);diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h index b195d1a..80f4b0e 100644 --- a/src/include/catalog/partition.h +++ b/src/include/catalog/partition.h @@ -89,4 +89,6 @@ extern int get_partition_for_tuple(PartitionDispatch *pd, TupleTableSlot *slot, EState *estate, Oid *failed_at); +extern Expr *convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc); +extern int get_next_hash_partition_index(Relation parent); #endif /* PARTITION_H */ diff --git a/src/include/catalog/pg_partitioned_table.h b/src/include/catalog/pg_partitioned_table.h index bdff36a..69e509c 100644 --- a/src/include/catalog/pg_partitioned_table.h +++ b/src/include/catalog/pg_partitioned_table.h @@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS char partstrat; /* partitioning strategy */ int16 partnatts; /* number of partition key columns */+ int16 partnparts;
+ Oid parthashfunc;
+
/*
* variable-length fields start here, but we allow direct access to
* partattrs via the C struct. That's because the first variable-length
@@ -49,6 +52,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
pg_node_tree partexprs; /* list of expressions in the partition key;
* one item for each zero entry in partattrs[] */
#endif
+
+
} FormData_pg_partitioned_table;/* ---------------- @@ -62,13 +67,15 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table; * compiler constants for pg_partitioned_table * ---------------- */ -#define Natts_pg_partitioned_table 7 +#define Natts_pg_partitioned_table 9 #define Anum_pg_partitioned_table_partrelid 1 #define Anum_pg_partitioned_table_partstrat 2 #define Anum_pg_partitioned_table_partnatts 3 -#define Anum_pg_partitioned_table_partattrs 4 -#define Anum_pg_partitioned_table_partclass 5 -#define Anum_pg_partitioned_table_partcollation 6 -#define Anum_pg_partitioned_table_partexprs 7 +#define Anum_pg_partitioned_table_partnparts 4 +#define Anum_pg_partitioned_table_parthashfunc 5 +#define Anum_pg_partitioned_table_partattrs 6 +#define Anum_pg_partitioned_table_partclass 7 +#define Anum_pg_partitioned_table_partcollation 8 +#define Anum_pg_partitioned_table_partexprs 9#endif /* PG_PARTITIONED_TABLE_H */ diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index 5afc3eb..1c3474f 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -730,11 +730,14 @@ typedef struct PartitionSpec NodeTag type; char *strategy; /* partitioning strategy ('list' or 'range') */ List *partParams; /* List of PartitionElems */ + int partnparts; + List *hashfunc; int location; /* token location, or -1 if unknown */ } PartitionSpec;#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'/* * PartitionBoundSpec - a partition bound specification diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index 985d650..0597939 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -180,6 +180,7 @@ PG_KEYWORD("greatest", GREATEST, COL_NAME_KEYWORD) PG_KEYWORD("group", GROUP_P, RESERVED_KEYWORD) PG_KEYWORD("grouping", GROUPING, COL_NAME_KEYWORD) PG_KEYWORD("handler", HANDLER, UNRESERVED_KEYWORD) +PG_KEYWORD("hash", HASH, UNRESERVED_KEYWORD) PG_KEYWORD("having", HAVING, RESERVED_KEYWORD) PG_KEYWORD("header", HEADER_P, UNRESERVED_KEYWORD) PG_KEYWORD("hold", HOLD, UNRESERVED_KEYWORD) @@ -291,6 +292,7 @@ PG_KEYWORD("parallel", PARALLEL, UNRESERVED_KEYWORD) PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD) PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD) PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD) +PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD) PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD) PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD) PG_KEYWORD("placing", PLACING, RESERVED_KEYWORD) diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h index a617a7c..660adfb 100644 --- a/src/include/utils/rel.h +++ b/src/include/utils/rel.h @@ -62,6 +62,9 @@ typedef struct PartitionKeyData Oid *partopcintype; /* OIDs of opclass declared input data types */ FmgrInfo *partsupfunc; /* lookup info for support funcs */+ int16 partnparts; /* number of hash partitions */ + Oid parthashfunc; /* OID of hash function */ + /* Partitioning collation per attribute */ Oid *partcollation;--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers--
Best regards,
Aleksander Alekseev--
Yugo Nagata <nagata@sraoss.co.jp>--
Best regards,
Aleksander Alekseev
--
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
On Wed, Mar 1, 2017 at 3:50 PM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
[....]
I Agree that it is unavoidable partitions number in modulo hashing,
but we can do in other hashing technique. Have you had thought about
Linear hashing[1] or Consistent hashing[2]? This will allow us to
add/drop
partition with minimal row moment. Thank you for your information of hash technique. I'll see them
and try to allowing the number of partitions to be changed.
Thanks for showing interest, I was also talking about this with Robert Haas
and
hacking on this, here is what we came up with this.
If we want to introduce hash partitioning without syntax contort and minimal
movement while changing hash partitions (ADD-DROP/ATTACH-DETACH operation),
at start I thought we could pick up linear hashing, because of in both the
hashing we might need to move approx tot_num_of_tuple/tot_num_of_partitions
tuples at adding new partition and no row moment required at dropping
partitioning.
With further thinking and talking through the idea of using linear hashing
with my team, we realized that has some problems specially during pg_dump
and pg_upgrade. Both a regular pg_dump and the binary-upgrade version of
pg_dump which is used by pg_restore need to maintain the identity of the
partitions. We can't rely on things like OID order which may be unstable, or
name order which might not match the order in which partitions were added.
So
somehow the partition position would need to be specified explicitly.
So later we came up with some syntax like this (just fyi, this doesn't add
any new keywords):
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 4, remainder 0);
create table foo2 partition of foo with (modulus 8, remainder 1); --
legal, modulus doesn't need to match
create table foo3 partition of foo with (modulus 8, remainder 4); --
illegal, overlaps foo1
Here we need to enforce a rule that every modulus must be a factor of the
next
larger modulus. So, 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. However, you could
simultaneously use modulus 4, modulus 8, modulus 16, and modulus 32 if you
wished, because each modulus is a factor of the next larger one. You could
also use modulus 10, modulus 20, and modulus 60. But you could not use
modulus
10, modulus 15, and modulus 60, because while both of the smaller module are
factors of 60, it is not true that each is a factor of the next.
Other advantages with this rule are:
1. Dropping (or detaching) and adding (or attaching) a partition can never
cause the rule to be violated.
2. We can easily build a tuple-routing data structure based on the largest
modulus.
For example: If the user has
partition 1 with (modulus 2, remainder 1),
partition 2 with (modulus 4, remainder 2),
partition 3 with (modulus 8, remainder 0) and
partition 4 with (modulus 8, remainder 4),
then we can build the following tuple routing array in the relcache:
== lookup table for hashvalue % 8 ==
0 => p3
1 => p1
2 => p2
3 => p1
4 => p4
5 => p1
6 => p2
7 => p1
3. It's also quite easy to test with a proposed new partition overlaps with
any
existing partition. Just build the mapping array and see if you ever end up
trying to assign a partition to a slot that's already been assigned to some
other partition.
We can still work on the proposed syntax - and I am open for suggestions.
One
more thought is to use FOR VALUES HAVING like:
CREATE TABLE foo1 PARTITION OF foo FOR VALUES HAVING (modulus 2, remainder
1);
But still more thoughts/inputs welcome here.
Attached patch implements former syntax, here is quick demonstration:
1.CREATE :
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 1);
create table foo2 partition of foo with (modulus 4, remainder 2);
create table foo3 partition of foo with (modulus 8, remainder 0);
create table foo4 partition of foo with (modulus 8, remainder 4);
2. Display parent table info:
postgres=# \d+ foo
Table "public.foo"
Column | Type | Collation | Nullable | Default | Storage | Stats
target | Description
--------+---------+-----------+----------+---------+----------+--------------+-------------
a | integer | | | | plain |
|
b | text | | | | extended |
|
Partition key: HASH (a)
Partitions: foo1 WITH (modulus 2, remainder 1),
foo2 WITH (modulus 4, remainder 2),
foo3 WITH (modulus 8, remainder 0),
foo4 WITH (modulus 8, remainder 4)
3. Display child table info:
postgres=# \d+ foo1
Table "public.foo1"
Column | Type | Collation | Nullable | Default | Storage | Stats
target | Description
--------+---------+-----------+----------+---------+----------+--------------+-------------
a | integer | | | | plain |
|
b | text | | | | extended |
|
Partition of: foo WITH (modulus 2, remainder 1)
4. INSERT:
postgres=# insert into foo select i, 'abc' from generate_series(1,10) i;
INSERT 0 10
postgres=# select tableoid::regclass as part, * from foo;
part | a | b
------+----+-----
foo1 | 3 | abc
foo1 | 4 | abc
foo1 | 7 | abc
foo1 | 10 | abc
foo2 | 1 | abc
foo2 | 2 | abc
foo2 | 9 | abc
foo3 | 6 | abc
foo4 | 5 | abc
foo4 | 8 | abc
(10 rows)
TODOs.
1. Maybe need some work in the CREATE TABLE .. PARTITION OF .. syntax.
2. Trim regression tests (if require).
3. Documentation
Thoughts/Comments?
Attachments:
hash-partitioning_another_design-v1.patchapplication/octet-stream; name=hash-partitioning_another_design-v1.patchDownload
From 7dde519faee8fdf41f3734f25b17a4ab434389d4 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Thu, 2 Mar 2017 13:32:34 +0530
Subject: [PATCH] Declarative hash partitioning
---
src/backend/catalog/partition.c | 569 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 63 +++-
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 | 46 ++-
src/backend/parser/parse_utilcmd.c | 28 +-
src/backend/utils/adt/ruleutils.c | 10 +
src/include/catalog/pg_proc.h | 3 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 231 ++++++++++++
src/test/regress/expected/create_table.out | 51 ++-
src/test/regress/expected/insert.out | 36 ++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 191 ++++++++++
src/test/regress/sql/create_table.sql | 37 +-
src/test/regress/sql/insert.sql | 30 ++
src/test/regress/sql/update.sql | 19 +
19 files changed, 1264 insertions(+), 87 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 4bcef58..ba1c727 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
@@ -76,16 +77,18 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
Datum **datums; /* Array of datum-tuples with key->partnatts
* datums each */
RangeDatumContent **content;/* what's contained in each range bound datum?
* (see the above enum); NULL for list
* partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ int *indexes; /* Partition indexes; in case of hash
+ * partitioned table array length will be
+ * value of largest modulus, and for others
+ * one entry per member of the datums array
+ * (plus one if range partitioned table) */
bool has_null; /* Is there a null-accepting partition? false
* for range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
@@ -97,6 +100,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
{
@@ -113,11 +124,13 @@ 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,
void *arg);
+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 Oid get_partition_operator(PartitionKey key, int col,
@@ -147,6 +160,11 @@ static void FormPartitionKeyDatum(PartitionDispatch pd,
Datum *values,
bool *isnull);
+static uint32 cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values,
+ bool *isnull);
+/* SQL-callable function for use in hash partition CHECK constraints */
+PG_FUNCTION_INFO_V1(satisfies_hash_partition);
+
/*
* RelationBuildPartitionDesc
* Form rel's partition descriptor
@@ -170,6 +188,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -236,7 +257,33 @@ 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 = 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;
@@ -463,6 +510,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));
+ memset(boundinfo->indexes, -1,
+ greatest_modulus * sizeof(int));
+
+ for (i = 0; i < nparts; i++)
+ {
+ int mod = hbounds[i]->modulus,
+ place = hbounds[i]->remainder;
+
+ boundinfo->datums[i] = (Datum *) palloc(2 *
+ sizeof(Datum));
+ boundinfo->datums[i][0] = Int32GetDatum(mod);
+ boundinfo->datums[i][1] = Int32GetDatum(place);
+ next_index = hbounds[i]->index;
+
+ while (place < greatest_modulus)
+ {
+ /* overlap? */
+ Assert(boundinfo->indexes[place] == -1);
+ boundinfo->indexes[place] = next_index;
+ place = place + mod;
+ }
+
+ mapping[i] = i;
+ pfree(hbounds[i]);
+ }
+ pfree(hbounds);
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
boundinfo->has_null = found_null;
@@ -616,53 +699,77 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
- for (j = 0; j < key->partnatts; j++)
+ /*
+ * Hash partition bound stores modulus and remainder at
+ * b1->datums[i][0] and b1->datums[i][0] position respectively.
+ */
+ for (i = 0; i < b1->ndatums; i++)
+ if (!(datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+ true, sizeof(int)) &&
+ datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+ true, sizeof(int))))
+ return false;
+
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound, and
+ * different kinds of infinities differ from each other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /* Non-finite bounds are equal without further examination. */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -684,6 +791,92 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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 place;
+
+ /*
+ * 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
+ * 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. However, you could simultaneously
+ * use modulus 4, modulus 8, modulus 16, and modulus 32 if
+ * you wished, because each modulus is a factor of the next
+ * larger one. You could also use modulus 10, modulus 20,
+ * and modulus 60. But you could not use modulus 10,
+ * modulus 15, and modulus 60 for the same reason.
+ */
+ {
+ int offset;
+ bool equal,
+ valid_bound = true;
+ int pmod, /* Previous largest modulus */
+ nmod; /* Next largest modulus */
+
+ /*
+ * 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)
+ {
+ nmod = DatumGetInt32(datums[0][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ else
+ {
+ pmod = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % pmod) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ nmod = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("every modulus must be factor of next largest modulus")));
+ }
+
+ greatest_modulus = DatumGetInt32(datums[ndatums - 1][0]);
+ place = spec->remainder;
+
+ if (place >= greatest_modulus)
+ place = place % greatest_modulus;
+
+ do
+ {
+ if (boundinfo->indexes[place] != -1)
+ {
+ overlap = true;
+ with = boundinfo->indexes[place];
+ break;
+ }
+ place = place + spec->modulus;
+ } while (place < greatest_modulus);
+ }
+
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
Assert(spec->strategy == PARTITION_STRATEGY_LIST);
@@ -906,6 +1099,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1149,6 +1347,93 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
/* Module-local functions */
/*
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder this function
+ * returns an expression Node for the partition table's CHECK constraint.
+ *
+ * For example, given a partition definition such as:
+ * CREATE TABLE simple_hash (pkey int, value char(10))
+ * PARTITION BY HASH (pkey, value);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash WITH (modulus 2, remainder 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash WITH (modulus 4, remainder 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash WITH (modulus 8, remainder 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash WITH (modulus 8, remainder 4);
+ *
+ * This function will return one of the following in the form of a
+ * subexpression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, pkey, value)
+ * for p_p2: satisfies_hash_partition(4, 2, pkey, value)
+ * for p_p3: satisfies_hash_partition(8, 0, pkey, value)
+ * for p_p4: satisfies_hash_partition(8, 4, pkey, value)
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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
+ {
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ args = lappend(args, keyCol);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns a list of expressions to use as a list partition's constraint.
@@ -1736,29 +2021,58 @@ get_partition_for_tuple(PartitionDispatch *pd,
cur_index = partdesc->boundinfo->null_index;
else
{
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
-
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
switch (key->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ {
+ PartitionBoundInfo boundinfo = partdesc->boundinfo;
+ int ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int modulus = DatumGetInt32(datum);
+ uint32 rowHash = cal_hash_value(key->partsupfunc,
+ key->partnatts,
+ values, isnull);
+
+ cur_index = boundinfo->indexes[rowHash % modulus];
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
- if (cur_offset >= 0 && equal)
- cur_index = partdesc->boundinfo->indexes[cur_offset];
- else
- cur_index = -1;
- break;
+ {
+ /* bsearch in partdesc->boundinfo */
+ bool equal = false;
+
+ 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;
+ }
case PARTITION_STRATEGY_RANGE:
+ {
+ bool equal = false;
- /*
- * Offset returned is such that the bound at offset is
- * found to be less or equal with the tuple. So, the bound
- * at offset+1 would be the upper bound.
- */
- cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
- break;
+ /* 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
+ * found to be less or equal with the tuple. So, the bound
+ * at offset+1 would be the upper bound.
+ */
+ cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
+ break;
+ }
default:
elog(ERROR, "unexpected partition strategy: %d",
@@ -1791,6 +2105,27 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * Used when sorting hash bounds across all hash modulus
+ * for hash partitioning
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+ PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+ PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+ int v1 = h1->modulus;
+ int v2 = h2->modulus;
+
+ if (v1 < v2)
+ return -1;
+ if (v1 > v2)
+ return 1;
+ if (v1 == v2 && h1->remainder != h2->remainder)
+ return (h1->remainder > h2->remainder) ? 1 : -1;
+ return 0;
+}
+
+/*
* qsort_partition_list_value_cmp
*
* Compare two list partition bound datums
@@ -1967,6 +2302,25 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
switch (key->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ {
+ PartitionBoundSpec *spec = (PartitionBoundSpec *) probe;
+ int mod = DatumGetInt32(bound_datums[0]);
+
+ if (mod < spec->modulus)
+ cmpval = -1;
+ else if (mod > spec->modulus)
+ cmpval = 1;
+ else if (mod == spec->modulus)
+ {
+ int rem = DatumGetInt32(bound_datums[1]);
+
+ cmpval = rem == spec->remainder ? 0 :
+ (rem < spec->remainder? -1 : 1);
+ }
+
+ break;
+ }
case PARTITION_STRATEGY_LIST:
cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
@@ -2050,3 +2404,110 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * Compute the hash value for given partition column values.
+ */
+static uint32
+cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /* rotate hash left 1 bit before mixing in the next column */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ {
+ Datum colHash;
+
+ Assert(OidIsValid(partsupfunc[i].fn_oid));
+
+ colHash = FunctionCall1(&partsupfunc[i], values[i]);
+ rowHash ^= DatumGetUInt32(colHash);
+ }
+ }
+
+ return rowHash;
+}
+
+/*
+ * satisfies_hash_partition
+ *
+ * This is a SQL-callable function for use in hash partition constraints;
+ * see get_qual_for_hash() for usage.
+ */
+Datum
+satisfies_hash_partition(PG_FUNCTION_ARGS)
+{
+ typedef struct ColumnsHashData
+ {
+ int n; /* allocated length of typentry[] */
+ TypeCacheEntry *typentry[PARTITION_MAX_KEYS];
+ } ColumnsHashData;
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum values[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ FmgrInfo partsupfunc[PARTITION_MAX_KEYS];
+ ColumnsHashData *my_extra;
+ uint32 rowHash = 0;
+
+ /*
+ * Cache hash function information, similar to how record_eq() caches
+ * equality operator information. (Perhaps no SQL syntax could cause
+ * PG_NARGS()/nkeys to change between calls through the same FmgrInfo.
+ * Checking nkeys here is just defensiveness.)
+ */
+ my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+ if (my_extra == NULL || my_extra->n != nkeys)
+ {
+ fcinfo->flinfo->fn_extra =
+ MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
+ offsetof(ColumnsHashData, typentry) +
+ sizeof(TypeCacheEntry *) * nkeys);
+ my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+ my_extra->n = nkeys;
+ }
+
+ /* Get TypeCacheEntry for each partition column. */
+ for (i = 0; i < nkeys; i++)
+ {
+ /* keys start from third argument of function. */
+ if (!PG_ARGISNULL(i + 2))
+ {
+ Oid valtype;
+
+ valtype = get_fn_expr_argtype(fcinfo->flinfo, (i + 2));
+ if (!OidIsValid(valtype))
+ elog(ERROR, "could not determine data type of satisfies_hash_partition() input");
+
+ /* Get the hash function. */
+ if (my_extra->typentry[i] == NULL ||
+ my_extra->typentry[i]->type_id != valtype)
+ {
+ my_extra->typentry[i] =
+ lookup_type_cache(valtype, TYPECACHE_HASH_PROC_FINFO);
+ if (!OidIsValid(my_extra->typentry[i]->hash_proc_finfo.fn_oid))
+ ereport(ERROR,
+ (errcode(ERRCODE_UNDEFINED_FUNCTION),
+ errmsg("could not identify a hash function for type %s",
+ format_type_be(valtype))));
+ }
+
+ values[i] = PG_GETARG_DATUM(i + 2);
+ isnull[i] = false;
+ partsupfunc[i] = my_extra->typentry[i]->hash_proc_finfo;
+ }
+ else
+ isnull[i] = true;
+ }
+
+ rowHash = cal_hash_value(partsupfunc, nkeys, values, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3cea220..8f6aa96 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -451,7 +451,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,
@@ -812,7 +812,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -12789,6 +12789,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -12844,7 +12846,8 @@ 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;
@@ -12989,27 +12992,49 @@ 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.
- */
- if (!pelem->opclass)
+ if (strategy == PARTITION_STRATEGY_HASH)
{
- partopclass[attn] = GetDefaultOpClass(atttype, BTREE_AM_OID);
+ /* Identify a hash opclass to use */
+ if (!pelem->opclass)
+ {
+ partopclass[attn] = GetDefaultOpClass(atttype, HASH_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 (!OidIsValid(partopclass[attn]))
+ 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
+ partopclass[attn] = ResolveOpClass(pelem->opclass,
+ atttype,
+ "hash",
+ HASH_AM_OID);
}
else
- partopclass[attn] = ResolveOpClass(pelem->opclass,
- atttype,
- "btree",
- BTREE_AM_OID);
+ {
+ /*
+ * Identify a btree opclass to use. Currently, we use only btree
+ * operators, which seems enough for list and range partitioning.
+ */
+ if (!pelem->opclass)
+ {
+ partopclass[attn] = GetDefaultOpClass(atttype, BTREE_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.")));
+ }
+ else
+ partopclass[attn] = ResolveOpClass(pelem->opclass,
+ atttype,
+ "btree",
+ BTREE_AM_OID);
+ }
attn++;
}
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 05d8538..250f6ab 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4257,6 +4257,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 d595cd7..3f4dd0f 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2746,6 +2746,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 b3802b4..2a39bb1 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3339,6 +3339,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 05bf2e9..d1aea69 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2293,6 +2293,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 e833b2e..cbac521 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -573,7 +573,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2563,8 +2564,35 @@ reloption_elem:
;
ForValues:
+ /* a HASH partition*/
+ WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+ n->strategy = PARTITION_STRATEGY_HASH;
+
+ foreach (lc, $3)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ n->remainder = defGetInt32(opt);
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("unrecognized hash partition bound specification \"%s\"",
+ opt->defname),
+ parser_errposition(opt->location)));
+ }
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2589,6 +2617,20 @@ ForValues:
}
;
+hash_partbound_elem:
+ NonReservedWord Iconst
+ {
+ $$ = makeDefElem($1, (Node *)makeInteger($2), @1);
+ }
+ ;
+
+hash_partbound:
+ hash_partbound_elem ',' hash_partbound_elem
+ {
+ $$ = list_make2($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 ff2bab6..de6c0f7 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3066,7 +3066,33 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("modulus must be greater than zero")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("remainder must be greater than or equal to zero")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("modulus 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 b27b77d..f85593c 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1507,6 +1507,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags)
switch (form->partstrat)
{
+ case PARTITION_STRATEGY_HASH:
+ appendStringInfo(&buf, "HASH");
+ break;
case PARTITION_STRATEGY_LIST:
appendStringInfo(&buf, "LIST");
break;
@@ -8454,6 +8457,13 @@ get_rule_expr(Node *node, deparse_context *context,
switch (spec->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ Assert(spec->modulus > 0 && spec->remainder >= 0);
+
+ appendStringInfo(buf, "WITH (modulus %d, remainder %d)",
+ spec->modulus, spec->remainder);
+ break;
+
case PARTITION_STRATEGY_LIST:
Assert(spec->listdatums != NIL);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4b9c6e7..a0cea73 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5361,6 +5361,9 @@ DESCR("pg_controldata init state information as a function");
DATA(insert OID = 3445 ( pg_import_system_collations PGNSP PGUID 12 100 0 0 0 f f f f t f v r 2 0 2278 "16 4089" _null_ _null_ "{if_not_exists,schema}" _null_ _null_ pg_import_system_collations _null_ _null_ _null_ ));
DESCR("import collations from operating system");
+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 2276" _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 5afc3eb..1bf00ec 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -728,11 +728,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -745,6 +747,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 9885fcb..c77563f 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3178,6 +3178,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,
@@ -3256,6 +3257,186 @@ 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
+-- check that target table is partitioned
+CREATE TABLE unparted (
+ a int
+);
+CREATE TABLE fail_part (like unparted);
+ALTER TABLE unparted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: "unparted" is not partitioned
+DROP TABLE unparted, fail_part;
+-- check that partition bound is compatible
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE fail_part (LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES FROM (1) TO (10);
+ERROR: invalid bound specification for a hash partition
+LINE 1: ...hash_parted ATTACH PARTITION fail_part FOR VALUES FROM (1) T...
+ ^
+DROP TABLE fail_part;
+-- check that the table being attached exists
+ALTER TABLE hash_parted ATTACH PARTITION nonexistant WITH (modulus 4, remainder 0);
+ERROR: relation "nonexistant" does not exist
+-- check ownership of the source table
+CREATE ROLE regress_test_me;
+CREATE ROLE regress_test_not_me;
+CREATE TABLE not_owned_by_me (LIKE hash_parted);
+ALTER TABLE not_owned_by_me OWNER TO regress_test_not_me;
+SET SESSION AUTHORIZATION regress_test_me;
+CREATE TABLE owned_by_me (
+ a int
+) PARTITION BY HASH (a);
+ALTER TABLE owned_by_me ATTACH PARTITION not_owned_by_me WITH (modulus 4, remainder 0);
+ERROR: must be owner of relation not_owned_by_me
+RESET SESSION AUTHORIZATION;
+DROP TABLE owned_by_me, not_owned_by_me;
+DROP ROLE regress_test_not_me;
+DROP ROLE regress_test_me;
+-- check that the table being attached is not part of regular inheritance
+CREATE TABLE parent (LIKE hash_parted);
+CREATE TABLE child () INHERITS (parent);
+ALTER TABLE hash_parted ATTACH PARTITION child WITH (modulus 4, remainder 1);
+ERROR: cannot attach inheritance child as partition
+ALTER TABLE hash_parted ATTACH PARTITION parent WITH (modulus 4, remainder 2);
+ERROR: cannot attach inheritance parent as partition
+DROP TABLE parent CASCADE;
+NOTICE: drop cascades to table child
+-- check any TEMP-ness
+CREATE TEMP TABLE temp_parted (a int) PARTITION BY HASH (a);
+CREATE TABLE perm_part (a int);
+ALTER TABLE temp_parted ATTACH PARTITION perm_part WITH (modulus 8, remainder 0);
+ERROR: cannot attach a permanent relation as partition of temporary relation "temp_parted"
+DROP TABLE temp_parted, perm_part;
+-- check that the table being attached is not a typed table
+CREATE TYPE mytype AS (a int);
+CREATE TABLE fail_part OF mytype;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 8, remainder 1);
+ERROR: cannot attach a typed table as partition
+DROP TYPE mytype CASCADE;
+NOTICE: drop cascades to table fail_part
+-- check existence (or non-existence) of oid column
+ALTER TABLE hash_parted SET WITH OIDS;
+CREATE TABLE fail_part (a int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: cannot attach table "fail_part" without OIDs as partition of table "hash_parted" with OIDs
+ALTER TABLE hash_parted SET WITHOUT OIDS;
+ALTER TABLE fail_part SET WITH OIDS;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: cannot attach table "fail_part" with OIDs as partition of table "hash_parted" without OIDs
+DROP TABLE fail_part;
+-- check that the table being attached has only columns present in the parent
+CREATE TABLE fail_part (like hash_parted, c int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: table "fail_part" contains column "c" not found in parent "hash_parted"
+DETAIL: New partition should contain only the columns present in parent.
+DROP TABLE fail_part;
+-- check that the table being attached has every column of the parent
+CREATE TABLE fail_part (a int NOT NULL);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: child table is missing column "b"
+DROP TABLE fail_part;
+-- check that columns match in type, collation and NOT NULL status
+CREATE TABLE fail_part (
+ b char(3),
+ a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: child table "fail_part" has different type for column "b"
+ALTER TABLE fail_part ALTER b TYPE char (2) COLLATE "POSIX";
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: child table "fail_part" has different collation for column "b"
+DROP TABLE fail_part;
+-- check that the table being attached has all constraints of the parent
+CREATE TABLE fail_part (
+ b char(2) COLLATE "C",
+ a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: child table is missing constraint "hcheck_a"
+-- check that the constraint matches in definition with parent's constraint
+ALTER TABLE fail_part ADD CONSTRAINT hcheck_a CHECK (a >= 0);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: child table "fail_part" has different definition for check constraint "hcheck_a"
+DROP TABLE fail_part;
+-- check the attributes and constraints after partition is attached
+CREATE TABLE hpart_1 (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_1 WITH (modulus 4, remainder 0);
+-- attislocal and conislocal are always false for merged attributes and constraints respectively.
+SELECT attislocal, attinhcount FROM pg_attribute WHERE attrelid = 'hpart_1'::regclass AND attnum > 0;
+ attislocal | attinhcount
+------------+-------------
+ f | 1
+ f | 1
+(2 rows)
+
+SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'hpart_1'::regclass AND conname = 'hcheck_a';
+ conislocal | coninhcount
+------------+-------------
+ f | 1
+(1 row)
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching list partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+-- check that the table being attached is not already a partition
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+ERROR: "hpart_2" is already a partition
+-- check that circular inheritance is not allowed
+ALTER TABLE hpart_5 ATTACH PARTITION hash_parted2 FOR VALUES IN ('b');
+ERROR: circular inheritance not allowed
+DETAIL: "hpart_5" is already a child of "hash_parted2".
+ALTER TABLE hash_parted2 ATTACH PARTITION hash_parted2 WITH (modulus 4, remainder 2);
+ERROR: circular inheritance not allowed
+DETAIL: "hash_parted2" is already a child of "hash_parted2".
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 0, remainder 1);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than zero
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 8, remainder 8);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than remainder
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 3, remainder 2);
+ERROR: invalid bound specification for a hash partition
+HINT: every modulus must be factor of next largest modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3267,12 +3448,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3290,12 +3478,32 @@ SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::r
(1 row)
DROP TABLE part_3_4;
+CREATE TABLE hpart_3 PARTITION OF hash_parted2(CONSTRAINT hcheck_a CHECK (a IN (3))) WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_3;
+SELECT attinhcount, attislocal FROM pg_attribute WHERE attrelid = 'hpart_3'::regclass AND attnum > 0;
+ attinhcount | attislocal
+-------------+------------
+ 0 | t
+ 0 | t
+(2 rows)
+
+SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'hpart_3'::regclass AND conname = 'hcheck_a';
+ coninhcount | conislocal
+-------------+------------
+ 0 | t
+(1 row)
+
+DROP TABLE hpart_3;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
ERROR: column must be added to child tables too
ALTER TABLE ONLY list_parted2 DROP COLUMN b;
ERROR: column must be dropped from child tables too
+ALTER TABLE ONLY hash_parted2 ADD COLUMN c int;
+ERROR: column must be added to child tables too
+ALTER TABLE ONLY hash_parted2 DROP COLUMN b;
+ERROR: column must be dropped from child tables too
-- cannot add a column to partition or drop an inherited one
ALTER TABLE part_2 ADD COLUMN c text;
ERROR: cannot add column to a partition
@@ -3313,17 +3521,32 @@ ALTER TABLE ONLY list_parted2 add constraint check_b check (b <> 'zz');
ERROR: constraint must be added to child tables too
ALTER TABLE list_parted2 add constraint check_b check (b <> 'zz') NO INHERIT;
ERROR: cannot add NO INHERIT constraint to partitioned table "list_parted2"
+ALTER TABLE ONLY hash_parted2 ALTER b SET NOT NULL;
+ERROR: constraint must be added to child tables too
+ALTER TABLE ONLY hash_parted2 add constraint hcheck_b check (b <> 'zz');
+ERROR: constraint must be added to child tables too
+ALTER TABLE hash_parted2 add constraint hcheck_b check (b <> 'zz') NO INHERIT;
+ERROR: cannot add NO INHERIT constraint to partitioned table "hash_parted2"
-- cannot drop inherited NOT NULL or check constraints from partition
ALTER TABLE list_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT check_a2 CHECK (a > 0);
ALTER TABLE part_2 ALTER b DROP NOT NULL;
ERROR: column "b" is marked NOT NULL in parent table
ALTER TABLE part_2 DROP CONSTRAINT check_a2;
ERROR: cannot drop inherited constraint "check_a2" of relation "part_2"
+ALTER TABLE hash_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT hcheck_a2 CHECK (a > 0);
+ALTER TABLE hpart_2 ALTER b DROP NOT NULL;
+ERROR: column "b" is marked NOT NULL in parent table
+ALTER TABLE hpart_2 DROP CONSTRAINT hcheck_a2;
+ERROR: cannot drop inherited constraint "hcheck_a2" of relation "hpart_2"
-- cannot drop NOT NULL or check constraints from *only* the parent
ALTER TABLE ONLY list_parted2 ALTER a DROP NOT NULL;
ERROR: constraint must be dropped from child tables too
ALTER TABLE ONLY list_parted2 DROP CONSTRAINT check_a2;
ERROR: constraint must be dropped from child tables too
+ALTER TABLE ONLY hash_parted2 ALTER a DROP NOT NULL;
+ERROR: constraint must be dropped from child tables too
+ALTER TABLE ONLY hash_parted2 DROP CONSTRAINT hcheck_a2;
+ERROR: constraint must be dropped from child tables too
-- check that a partition cannot participate in regular inheritance
CREATE TABLE inh_test () INHERITS (part_2);
ERROR: cannot inherit from partition "part_2"
@@ -3339,10 +3562,18 @@ ALTER TABLE list_parted2 DROP COLUMN b;
ERROR: cannot drop column named in partition key
ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
ERROR: cannot alter type of column named in partition key
+-- similarly, hpart_5 which is hash_parted2's partition,
+-- is partitioned on b;
+ALTER TABLE hash_parted2 DROP COLUMN b;
+ERROR: cannot drop column named in partition key
+ALTER TABLE hash_parted2 ALTER COLUMN b TYPE text;
+ERROR: cannot alter type of column named in partition key
-- cleanup: avoid using CASCADE
DROP TABLE list_parted, part_1;
DROP TABLE list_parted2, part_2, part_5, part_5_a;
DROP TABLE range_parted, part1, part2;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 20eb3d3..4ccb524 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
@@ -505,6 +500,21 @@ 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 (unbounded);
ERROR: cannot specify NULL in range bound
+-- 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 WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted 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 WITH (modulus 25, remainder 2);
+ERROR: invalid bound specification for a hash partition
+HINT: every modulus must be factor of next largest 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',...
+ ^
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
CREATE TABLE unparted (
@@ -512,6 +522,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 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 (
@@ -519,6 +531,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -526,6 +540,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -533,6 +549,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted WITH (modulu...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -591,6 +611,25 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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 WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 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 WITH (modulus 0, remainder 1);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than zero
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 8, remainder 8);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -674,3 +713,5 @@ DROP TABLE range_parted;
DROP TABLE list_parted2, part_ab, part_null_z;
DROP TABLE range_parted2, part0, part1, part2, part3;
DROP TABLE range_parted3, part00, part10, part11, part12;
+DROP TABLE hash_parted, hpart_1, hpart_2;
+DROP TABLE hash_parted2, h2part_1, h2part_2, h2part_3, h2part_4;
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index 81af3ef..48db21d 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,11 +313,47 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart4 values ('b', 21);
+ERROR: new row for relation "hpart4" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart4 values ('a', 10);
+ERROR: new row for relation "hpart4" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart4 values ('c', 6);
+-- fail
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- ok
+insert into hpart3 values (null);
+insert into hpart1 values (1);
-- cleanup
drop table part1, part2, part3, part4, range_parted;
drop table part_ee_ff3_1, part_ee_ff3_2, part_ee_ff1, part_ee_ff2, part_ee_ff3;
drop table part_ee_ff, part_gg2_2, part_gg2_1, part_gg2, part_gg1, part_gg;
drop table part_aa_bb, part_cc_dd, part_null, list_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
-- 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 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 a1e9255..c41e26e 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,6 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted cascade;
NOTICE: drop cascades to 4 other objects
@@ -225,3 +245,4 @@ DETAIL: drop cascades to table part_a_1_a_10
drop cascades to table part_a_10_a_20
drop cascades to table part_b_1_b_10
drop cascades to table part_b_10_b_20
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index f7b754f..e545102 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2050,6 +2050,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 (
@@ -2135,6 +2136,166 @@ 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
+
+-- check that target table is partitioned
+CREATE TABLE unparted (
+ a int
+);
+CREATE TABLE fail_part (like unparted);
+ALTER TABLE unparted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE unparted, fail_part;
+
+-- check that partition bound is compatible
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE fail_part (LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES FROM (1) TO (10);
+DROP TABLE fail_part;
+
+-- check that the table being attached exists
+ALTER TABLE hash_parted ATTACH PARTITION nonexistant WITH (modulus 4, remainder 0);
+
+-- check ownership of the source table
+CREATE ROLE regress_test_me;
+CREATE ROLE regress_test_not_me;
+CREATE TABLE not_owned_by_me (LIKE hash_parted);
+ALTER TABLE not_owned_by_me OWNER TO regress_test_not_me;
+SET SESSION AUTHORIZATION regress_test_me;
+CREATE TABLE owned_by_me (
+ a int
+) PARTITION BY HASH (a);
+ALTER TABLE owned_by_me ATTACH PARTITION not_owned_by_me WITH (modulus 4, remainder 0);
+RESET SESSION AUTHORIZATION;
+DROP TABLE owned_by_me, not_owned_by_me;
+DROP ROLE regress_test_not_me;
+DROP ROLE regress_test_me;
+
+-- check that the table being attached is not part of regular inheritance
+CREATE TABLE parent (LIKE hash_parted);
+CREATE TABLE child () INHERITS (parent);
+ALTER TABLE hash_parted ATTACH PARTITION child WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION parent WITH (modulus 4, remainder 2);
+DROP TABLE parent CASCADE;
+
+-- check any TEMP-ness
+CREATE TEMP TABLE temp_parted (a int) PARTITION BY HASH (a);
+CREATE TABLE perm_part (a int);
+ALTER TABLE temp_parted ATTACH PARTITION perm_part WITH (modulus 8, remainder 0);
+DROP TABLE temp_parted, perm_part;
+
+-- check that the table being attached is not a typed table
+CREATE TYPE mytype AS (a int);
+CREATE TABLE fail_part OF mytype;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 8, remainder 1);
+DROP TYPE mytype CASCADE;
+
+-- check existence (or non-existence) of oid column
+ALTER TABLE hash_parted SET WITH OIDS;
+CREATE TABLE fail_part (a int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+
+ALTER TABLE hash_parted SET WITHOUT OIDS;
+ALTER TABLE fail_part SET WITH OIDS;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that the table being attached has only columns present in the parent
+CREATE TABLE fail_part (like hash_parted, c int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that the table being attached has every column of the parent
+CREATE TABLE fail_part (a int NOT NULL);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that columns match in type, collation and NOT NULL status
+CREATE TABLE fail_part (
+ b char(3),
+ a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ALTER TABLE fail_part ALTER b TYPE char (2) COLLATE "POSIX";
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that the table being attached has all constraints of the parent
+CREATE TABLE fail_part (
+ b char(2) COLLATE "C",
+ a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+
+-- check that the constraint matches in definition with parent's constraint
+ALTER TABLE fail_part ADD CONSTRAINT hcheck_a CHECK (a >= 0);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check the attributes and constraints after partition is attached
+CREATE TABLE hpart_1 (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_1 WITH (modulus 4, remainder 0);
+-- attislocal and conislocal are always false for merged attributes and constraints respectively.
+SELECT attislocal, attinhcount FROM pg_attribute WHERE attrelid = 'hpart_1'::regclass AND attnum > 0;
+SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'hpart_1'::regclass AND conname = 'hcheck_a';
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching list partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+
+-- check that the table being attached is not already a partition
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+
+-- check that circular inheritance is not allowed
+ALTER TABLE hpart_5 ATTACH PARTITION hash_parted2 FOR VALUES IN ('b');
+ALTER TABLE hash_parted2 ATTACH PARTITION hash_parted2 WITH (modulus 4, remainder 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2146,12 +2307,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2159,11 +2325,18 @@ SELECT attinhcount, attislocal FROM pg_attribute WHERE attrelid = 'part_3_4'::re
SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::regclass AND conname = 'check_a';
DROP TABLE part_3_4;
+CREATE TABLE hpart_3 PARTITION OF hash_parted2(CONSTRAINT hcheck_a CHECK (a IN (3))) WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_3;
+SELECT attinhcount, attislocal FROM pg_attribute WHERE attrelid = 'hpart_3'::regclass AND attnum > 0;
+SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'hpart_3'::regclass AND conname = 'hcheck_a';
+DROP TABLE hpart_3;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY hash_parted2 ADD COLUMN c int;
+ALTER TABLE ONLY hash_parted2 DROP COLUMN b;
-- cannot add a column to partition or drop an inherited one
ALTER TABLE part_2 ADD COLUMN c text;
@@ -2178,15 +2351,26 @@ ALTER TABLE ONLY list_parted2 ALTER b SET NOT NULL;
ALTER TABLE ONLY list_parted2 add constraint check_b check (b <> 'zz');
ALTER TABLE list_parted2 add constraint check_b check (b <> 'zz') NO INHERIT;
+ALTER TABLE ONLY hash_parted2 ALTER b SET NOT NULL;
+ALTER TABLE ONLY hash_parted2 add constraint hcheck_b check (b <> 'zz');
+ALTER TABLE hash_parted2 add constraint hcheck_b check (b <> 'zz') NO INHERIT;
+
-- cannot drop inherited NOT NULL or check constraints from partition
ALTER TABLE list_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT check_a2 CHECK (a > 0);
ALTER TABLE part_2 ALTER b DROP NOT NULL;
ALTER TABLE part_2 DROP CONSTRAINT check_a2;
+ALTER TABLE hash_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT hcheck_a2 CHECK (a > 0);
+ALTER TABLE hpart_2 ALTER b DROP NOT NULL;
+ALTER TABLE hpart_2 DROP CONSTRAINT hcheck_a2;
+
-- cannot drop NOT NULL or check constraints from *only* the parent
ALTER TABLE ONLY list_parted2 ALTER a DROP NOT NULL;
ALTER TABLE ONLY list_parted2 DROP CONSTRAINT check_a2;
+ALTER TABLE ONLY hash_parted2 ALTER a DROP NOT NULL;
+ALTER TABLE ONLY hash_parted2 DROP CONSTRAINT hcheck_a2;
+
-- check that a partition cannot participate in regular inheritance
CREATE TABLE inh_test () INHERITS (part_2);
CREATE TABLE inh_test (LIKE part_2);
@@ -2199,10 +2383,17 @@ ALTER TABLE part_2 INHERIT inh_test;
ALTER TABLE list_parted2 DROP COLUMN b;
ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
+-- similarly, hpart_5 which is hash_parted2's partition,
+-- is partitioned on b;
+ALTER TABLE hash_parted2 DROP COLUMN b;
+ALTER TABLE hash_parted2 ALTER COLUMN b TYPE text;
+
-- cleanup: avoid using CASCADE
DROP TABLE list_parted, part_1;
DROP TABLE list_parted2, part_2, part_5, part_5_a;
DROP TABLE range_parted, part1, part2;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 f41dd71..268d77d 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
@@ -473,6 +468,17 @@ 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 (unbounded);
+-- 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 WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted 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 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');
+
-- check if compatible with the specified parent
-- cannot create as partition of a non-partitioned table
@@ -480,6 +486,7 @@ CREATE TABLE unparted (
a int
);
CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted WITH (modulus 2, remainder 1);
DROP TABLE unparted;
-- cannot create a permanent rel as partition of a temp rel
@@ -487,6 +494,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -494,6 +502,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -502,6 +511,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -553,6 +563,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, unbounded) TO (1, unbounded);
+-- 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 WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 2, remainder 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 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 WITH (modulus 8, remainder 8);
+
-- check schema propagation from parent
CREATE TABLE parted (
@@ -602,3 +627,5 @@ DROP TABLE range_parted;
DROP TABLE list_parted2, part_ab, part_null_z;
DROP TABLE range_parted2, part0, part1, part2, part3;
DROP TABLE range_parted3, part00, part10, part11, part12;
+DROP TABLE hash_parted, hpart_1, hpart_2;
+DROP TABLE hash_parted2, h2part_1, h2part_2, h2part_3, h2part_4;
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index 454e1ce..36ebb45 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,11 +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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart4 values ('b', 21);
+insert into hpart4 values ('a', 10);
+-- ok
+insert into hpart4 values ('c', 6);
+
+-- fail
+insert into hpart1 values (null);
+
+-- ok
+insert into hpart3 values (null);
+insert into hpart1 values (1);
+
-- cleanup
drop table part1, part2, part3, part4, range_parted;
drop table part_ee_ff3_1, part_ee_ff3_2, part_ee_ff1, part_ee_ff2, part_ee_ff3;
drop table part_ee_ff, part_gg2_2, part_gg2_1, part_gg2, part_gg1, part_gg;
drop table part_aa_bb, part_cc_dd, part_null, list_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
-- 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/update.sql b/src/test/regress/sql/update.sql
index d7721ed..4fc0b21 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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 cascade;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Thu, 2 Mar 2017 18:33:42 +0530
amul sul <sulamul@gmail.com> wrote:
Thank you for the patch. This is very interesting. I'm going to look
into your code and write a feedback later.
On Wed, Mar 1, 2017 at 3:50 PM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
[....]
I Agree that it is unavoidable partitions number in modulo hashing,
but we can do in other hashing technique. Have you had thought about
Linear hashing[1] or Consistent hashing[2]? This will allow us to
add/drop
partition with minimal row moment. Thank you for your information of hash technique. I'll see them
and try to allowing the number of partitions to be changed.
Thanks for showing interest, I was also talking about this with Robert Haas
and
hacking on this, here is what we came up with this.If we want to introduce hash partitioning without syntax contort and minimal
movement while changing hash partitions (ADD-DROP/ATTACH-DETACH operation),
at start I thought we could pick up linear hashing, because of in both the
hashing we might need to move approx tot_num_of_tuple/tot_num_of_partitions
tuples at adding new partition and no row moment required at dropping
partitioning.With further thinking and talking through the idea of using linear hashing
with my team, we realized that has some problems specially during pg_dump
and pg_upgrade. Both a regular pg_dump and the binary-upgrade version of
pg_dump which is used by pg_restore need to maintain the identity of the
partitions. We can't rely on things like OID order which may be unstable, or
name order which might not match the order in which partitions were added.
So
somehow the partition position would need to be specified explicitly.So later we came up with some syntax like this (just fyi, this doesn't add
any new keywords):create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 4, remainder 0);
create table foo2 partition of foo with (modulus 8, remainder 1); --
legal, modulus doesn't need to match
create table foo3 partition of foo with (modulus 8, remainder 4); --
illegal, overlaps foo1Here we need to enforce a rule that every modulus must be a factor of the
next
larger modulus. So, 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. However, you could
simultaneously use modulus 4, modulus 8, modulus 16, and modulus 32 if you
wished, because each modulus is a factor of the next larger one. You could
also use modulus 10, modulus 20, and modulus 60. But you could not use
modulus
10, modulus 15, and modulus 60, because while both of the smaller module are
factors of 60, it is not true that each is a factor of the next.Other advantages with this rule are:
1. Dropping (or detaching) and adding (or attaching) a partition can never
cause the rule to be violated.2. We can easily build a tuple-routing data structure based on the largest
modulus.For example: If the user has
partition 1 with (modulus 2, remainder 1),
partition 2 with (modulus 4, remainder 2),
partition 3 with (modulus 8, remainder 0) and
partition 4 with (modulus 8, remainder 4),then we can build the following tuple routing array in the relcache:
== lookup table for hashvalue % 8 ==
0 => p3
1 => p1
2 => p2
3 => p1
4 => p4
5 => p1
6 => p2
7 => p13. It's also quite easy to test with a proposed new partition overlaps with
any
existing partition. Just build the mapping array and see if you ever end up
trying to assign a partition to a slot that's already been assigned to some
other partition.We can still work on the proposed syntax - and I am open for suggestions.
One
more thought is to use FOR VALUES HAVING like:
CREATE TABLE foo1 PARTITION OF foo FOR VALUES HAVING (modulus 2, remainder
1);But still more thoughts/inputs welcome here.
Attached patch implements former syntax, here is quick demonstration:
1.CREATE :
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 1);
create table foo2 partition of foo with (modulus 4, remainder 2);
create table foo3 partition of foo with (modulus 8, remainder 0);
create table foo4 partition of foo with (modulus 8, remainder 4);2. Display parent table info:
postgres=# \d+ foo
Table "public.foo"
Column | Type | Collation | Nullable | Default | Storage | Stats
target | Description
--------+---------+-----------+----------+---------+----------+--------------+-------------
a | integer | | | | plain |
|
b | text | | | | extended |
|
Partition key: HASH (a)
Partitions: foo1 WITH (modulus 2, remainder 1),
foo2 WITH (modulus 4, remainder 2),
foo3 WITH (modulus 8, remainder 0),
foo4 WITH (modulus 8, remainder 4)3. Display child table info:
postgres=# \d+ foo1
Table "public.foo1"
Column | Type | Collation | Nullable | Default | Storage | Stats
target | Description
--------+---------+-----------+----------+---------+----------+--------------+-------------
a | integer | | | | plain |
|
b | text | | | | extended |
|
Partition of: foo WITH (modulus 2, remainder 1)4. INSERT:
postgres=# insert into foo select i, 'abc' from generate_series(1,10) i;
INSERT 0 10postgres=# select tableoid::regclass as part, * from foo;
part | a | b
------+----+-----
foo1 | 3 | abc
foo1 | 4 | abc
foo1 | 7 | abc
foo1 | 10 | abc
foo2 | 1 | abc
foo2 | 2 | abc
foo2 | 9 | abc
foo3 | 6 | abc
foo4 | 5 | abc
foo4 | 8 | abc
(10 rows)TODOs.
1. Maybe need some work in the CREATE TABLE .. PARTITION OF .. syntax.
2. Trim regression tests (if require).
3. DocumentationThoughts/Comments?
--
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
On 2 March 2017 at 13:03, amul sul <sulamul@gmail.com> wrote:
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 4, remainder 0);
create table foo2 partition of foo with (modulus 8, remainder 1); -- legal,
modulus doesn't need to match
create table foo3 partition of foo with (modulus 8, remainder 4); --
illegal, overlaps foo1
Instead of using modulus, why not just divide up the range of hash
keys using ranges? That should be just as good for a good hash
function (effectively using the high bits instead of the low bits of
the hash value). And it would mean you could reuse the machinery for
list partitioning for partition exclusion.
It also has the advantage that it's easier to see how to add more
partitions. You just split all the ranges and (and migrate the
data...). There's even the possibility of having uneven partitions if
you have a data distribution skew -- which can happen even if you have
a good hash function. In a degenerate case you could have a partition
for a single hash of a particularly common value then a reasonable
number of partitions for the remaining hash ranges.
--
greg
--
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, Mar 3, 2017 at 5:00 PM, Greg Stark <stark@mit.edu> wrote:
On 2 March 2017 at 13:03, amul sul <sulamul@gmail.com> wrote:
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 4, remainder 0);
create table foo2 partition of foo with (modulus 8, remainder 1); --legal,
modulus doesn't need to match
create table foo3 partition of foo with (modulus 8, remainder 4); --
illegal, overlaps foo1Instead of using modulus, why not just divide up the range of hash
keys using ranges?
That should be just as good for a good hash
function (effectively using the high bits instead of the low bits of
the hash value). And it would mean you could reuse the machinery for
list partitioning for partition exclusion.
It also has the advantage that it's easier to see how to add more
partitions. You just split all the ranges and (and migrate the
data...). There's even the possibility of having uneven partitions if
you have a data distribution skew -- which can happen even if you have
a good hash function. In a degenerate case you could have a partition
for a single hash of a particularly common value then a reasonable
number of partitions for the remaining hash ranges.
Initially
we
had
to have
somewhat similar thought to make a range of hash
values for
each partition, using the same half-open interval syntax we use in general:
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo for values from (0) to (1073741824);
create table foo2 partition of foo for values from (1073741824) to
(-2147483648);
create table foo3 partition of foo for values from (-2147483648) to
(-1073741824);
create table foo4 partition of foo for values from (-1073741824) to (0);
That's really nice for the system, but not so much for the users. The
system can
now generate each partition constraint correctly immediately upon seeing
the SQL
statement for the corresponding table, which is very desirable. However,
users are
not likely to know that the magic numbers to distribute keys equally across
four
partitions are 1073741824, -2147483648, and -1073741824.
So it's pretty
user-unfriendly.
Regards,
Amul
On 3/3/17 8:33 AM, amul sul wrote:
On Fri, Mar 3, 2017 at 5:00 PM, Greg Stark <stark@mit.edu
It also has the advantage that it's easier to see how to add more
partitions. You just split all the ranges and (and migrate the
data...). There's even the possibility of having uneven partitions if
you have a data distribution skew -- which can happen even if you have
a good hash function. In a degenerate case you could have a partition
for a single hash of a particularly common value then a reasonable
number of partitions for the remaining hash ranges.Initially
we
had
to have
somewhat similar thought to make a range of hash
values for
each partition, using the same half-open interval syntax we use in general:
<...>
So it's pretty
user-unfriendly.
This patch is marked as POC and after a read-through I agree that's
exactly what it is. As such, I'm not sure it belongs in the last
commitfest. Furthermore, there has not been any activity or a new patch
in a while and we are halfway through the CF.
Please post an explanation for the delay and a schedule for the new
patch. If no patch or explanation is posted by 2017-03-17 AoE I will
mark this submission "Returned with Feedback".
--
-David
david@pgmasters.net
--
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, Mar 14, 2017 at 10:08 AM, David Steele <david@pgmasters.net> wrote:
This patch is marked as POC and after a read-through I agree that's
exactly what it is.
Just out of curiosity, were you looking at Nagata-san's patch, or Amul's?
As such, I'm not sure it belongs in the last
commitfest. Furthermore, there has not been any activity or a new patch
in a while and we are halfway through the CF.Please post an explanation for the delay and a schedule for the new
patch. If no patch or explanation is posted by 2017-03-17 AoE I will
mark this submission "Returned with Feedback".
Regrettably, I do think it's too late to squeeze hash partitioning
into v10, but I plan to try to get something committed for v11. I was
heavily involved in the design of Amul's patch, and I think that
design solves several problems that would be an issue for us if we did
as Nagata-san is proposing. For example, he proposed this:
CREATE TABLE h1 PARTITION OF h;
CREATE TABLE h2 PARTITION OF h;
CREATE TABLE h3 PARTITION OF h;
That looks OK if you are thinking of typing this in interactively, but
if you're doing a pg_dump, maybe with --binary-upgrade, you don't want
the meaning of a series of nearly-identical SQL commands to depend on
the dump ordering. You want it to be explicit in the SQL command
which partition is which, and Amul's patch solves that problem. Also,
Nagata-san's proposal doesn't provide any way to increase the number
of partitions later, and Amul's approach gives you some options there.
I'm not sure those options are as good as we'd like them to be, and if
not then we may need to revise the approach, but I'm pretty sure
having no strategy at all for changing the partition count is not good
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 3/15/17 12:25 PM, Robert Haas wrote:
On Tue, Mar 14, 2017 at 10:08 AM, David Steele <david@pgmasters.net> wrote:
This patch is marked as POC and after a read-through I agree that's
exactly what it is.Just out of curiosity, were you looking at Nagata-san's patch, or Amul's?
Both - what I was looking for was some kind of reconciliation between
the two patches and I didn't find that. It seemed from the thread that
Yugo intended to pull Amul's changes/idea into his patch.
As such, I'm not sure it belongs in the last
commitfest. Furthermore, there has not been any activity or a new patch
in a while and we are halfway through the CF.Please post an explanation for the delay and a schedule for the new
patch. If no patch or explanation is posted by 2017-03-17 AoE I will
mark this submission "Returned with Feedback".Regrettably, I do think it's too late to squeeze hash partitioning
into v10, but I plan to try to get something committed for v11.
It would certainly be a nice feature to have.
I was
heavily involved in the design of Amul's patch, and I think that
design solves several problems that would be an issue for us if we did
as Nagata-san is proposing. For example, he proposed this:CREATE TABLE h1 PARTITION OF h;
CREATE TABLE h2 PARTITION OF h;
CREATE TABLE h3 PARTITION OF h;That looks OK if you are thinking of typing this in interactively, but
if you're doing a pg_dump, maybe with --binary-upgrade, you don't want
the meaning of a series of nearly-identical SQL commands to depend on
the dump ordering. You want it to be explicit in the SQL command
which partition is which, and Amul's patch solves that problem.
OK, it wasn't clear to me that this was the case because of the stated
user-unfriendliness.
Also,
Nagata-san's proposal doesn't provide any way to increase the number
of partitions later, and Amul's approach gives you some options there.
I'm not sure those options are as good as we'd like them to be, and if
not then we may need to revise the approach, but I'm pretty sure
having no strategy at all for changing the partition count is not good
enough.
Agreed. Perhaps both types of syntax should be supported, one that is
friendly to users and one that is precise for dump tools and those who
care get in the weeds.
--
-David
david@pgmasters.net
--
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, Mar 15, 2017 at 12:39 PM, David Steele <david@pgmasters.net> wrote:
Agreed. Perhaps both types of syntax should be supported, one that is
friendly to users and one that is precise for dump tools and those who
care get in the weeds.
Eventually, sure. For the first version, I want to skip the friendly
syntax and just add the necessary syntax. That makes it easier to
make sure that pg_dump and everything are working the way you want.
Range and list partitioning could potentially grow convenience syntax
around partition creation, too, but that wasn't essential for the
first patch, so we cut it.
--
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, 14 Mar 2017 10:08:14 -0400
David Steele <david@pgmasters.net> wrote:
Please post an explanation for the delay and a schedule for the new
patch. If no patch or explanation is posted by 2017-03-17 AoE I will
mark this submission "Returned with Feedback".
I am sorry for my late response. I had not a enough time because I had a
business trip and was busy for other works.
I agree that fixing the number of partitions is bad and a way
to increase or decrease partitions should be provided. I also think
using linear hashing would be good as Amul is mentioning, but I
have not implemented it in my patch yet.
I also understanded that my design has a problem during pg_dump and
pg_upgrade, and that some information to identify the partition
is required not depending the command order. However, I feel that
Amul's design is a bit complicated with the rule to specify modulus.
I think we can use simpler syntax, for example, as below.
CREATE TABLE h1 PARTITION OF h FOR (0);
CREATE TABLE h2 PARTITION OF h FOR (1);
CREATE TABLE h3 PARTITION OF h FOR (2);
If user want to user any complicated partitioning rule, it can be defined
by specifying a user-defined hash function at creating partitioned table.
If the hash function is omitted, we will be able to use default hash
operator class as well as in Amul's patch.
Attached is the updated patch taking the comments from Aleksander and Rushabh.
HASH keyword and unnecessary spaces are removed, and some comments are added.
Thanks,
--
Yugo Nagata <nagata@sraoss.co.jp>
Attachments:
hash_partition.patch.v2application/octet-stream; name=hash_partition.patch.v2Download
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 41c0056..257c6b8 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -55,6 +55,7 @@
#include "catalog/pg_tablespace.h"
#include "catalog/pg_type.h"
#include "catalog/pg_type_fn.h"
+#include "catalog/pg_proc.h"
#include "catalog/storage.h"
#include "catalog/storage_xlog.h"
#include "commands/tablecmds.h"
@@ -3074,7 +3075,7 @@ StorePartitionKey(Relation rel,
AttrNumber *partattrs,
List *partexprs,
Oid *partopclass,
- Oid *partcollation)
+ Oid *partcollation, int16 partnparts, Oid hashfunc)
{
int i;
int2vector *partattrs_vec;
@@ -3121,6 +3122,8 @@ StorePartitionKey(Relation rel,
values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel));
values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy);
values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts);
+ values[Anum_pg_partitioned_table_partnparts - 1] = Int16GetDatum(partnparts);
+ values[Anum_pg_partitioned_table_parthashfunc - 1] = ObjectIdGetDatum(hashfunc);
values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec);
values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec);
values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec);
@@ -3152,6 +3155,16 @@ StorePartitionKey(Relation rel,
recordDependencyOn(&myself, &referenced, DEPENDENCY_NORMAL);
}
+ /* Record dependencies for hash function */
+ if (hashfunc != InvalidOid)
+ {
+ referenced.classId = ProcedureRelationId;
+ referenced.objectId = hashfunc;
+ referenced.objectSubId = 0;
+
+ recordDependencyOn(&myself, &referenced, DEPENDENCY_NORMAL);
+ }
+
/*
* Anything mentioned in the expressions. We must ignore the column
* references, which will depend on the table itself; there is no separate
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index e01ef86..63d9c8e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -36,6 +36,8 @@
#include "optimizer/clauses.h"
#include "optimizer/planmain.h"
#include "optimizer/var.h"
+#include "parser/parse_func.h"
+#include "parser/parse_oper.h"
#include "rewrite/rewriteManip.h"
#include "storage/lmgr.h"
#include "utils/array.h"
@@ -120,6 +122,7 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
+static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec);
static Oid get_partition_operator(PartitionKey key, int col,
StrategyNumber strategy, bool *need_relabel);
static List *generate_partition_qual(Relation rel);
@@ -229,7 +232,8 @@ 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_LIST ||
+ key->strategy == PARTITION_STRATEGY_HASH)
{
List *non_null_values = NIL;
@@ -244,7 +248,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
+ if (spec->strategy != key->strategy)
elog(ERROR, "invalid strategy in partition bound spec");
foreach(c, spec->listdatums)
@@ -457,6 +461,7 @@ RelationBuildPartitionDesc(Relation rel)
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
{
boundinfo->has_null = found_null;
boundinfo->indexes = (int *) palloc(ndatums * sizeof(int));
@@ -822,6 +827,18 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
break;
}
+ case PARTITION_STRATEGY_HASH:
+ {
+ Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+
+ if (partdesc->nparts + 1 > key->partnparts)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+ errmsg("cannot create hash partition more than %d for %s",
+ key->partnparts, RelationGetRelationName(parent))));
+ break;
+ }
+
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
@@ -909,6 +926,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
my_qual = get_qual_for_range(key, spec);
break;
+ case PARTITION_STRATEGY_HASH:
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
+ my_qual = get_qual_for_hash(key, spec);
+ break;
+
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) key->strategy);
@@ -1139,6 +1161,84 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
return pd;
}
+/*
+ * convert_expr_for_hash
+ *
+ * Converts a expr for a hash partition's constraint.
+ * expr is converted into 'abs(hashfunc(expr)) % npart".
+ *
+ * npart: number of partitions
+ * hashfunc: OID of hash function
+ */
+Expr *
+convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc)
+{
+ FuncExpr *func,
+ *abs;
+ Expr *modexpr;
+ Oid modoid;
+ Oid int4oid[1] = {INT4OID};
+
+ ParseState *pstate = make_parsestate(NULL);
+ Value *val_npart = makeInteger(npart);
+ Node *const_npart = (Node *) make_const(pstate, val_npart, -1);
+
+ /* hash function */
+ func = makeFuncExpr(hashfunc,
+ INT4OID,
+ list_make1(expr),
+ 0,
+ 0,
+ COERCE_EXPLICIT_CALL);
+
+ /* Abs */
+ abs = makeFuncExpr(LookupFuncName(list_make1(makeString("abs")), 1, int4oid, false),
+ INT4OID,
+ list_make1(func),
+ 0,
+ 0,
+ COERCE_EXPLICIT_CALL);
+
+ /* modulo by npart */
+ modoid = LookupOperName(pstate, list_make1(makeString("%")), INT4OID, INT4OID, false, -1);
+ modexpr = make_opclause(modoid, INT4OID, false, (Expr*)abs, (Expr*)const_npart, 0, 0);
+
+ return modexpr;
+}
+
+
+/*
+ * get_next_hash_partition_index
+ *
+ * Returns the minimal index which is not used for hash partition.
+ */
+int
+get_next_hash_partition_index(Relation parent)
+{
+ PartitionKey key = RelationGetPartitionKey(parent);
+ PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+
+ int i;
+ bool *used = palloc0(sizeof(int) * key->partnparts);
+
+ /* mark used for existing partition indexs */
+ for (i = 0; i < partdesc->boundinfo->ndatums; i++)
+ {
+ Datum* datum = partdesc->boundinfo->datums[i];
+ int idx = DatumGetInt16(datum[0]);
+
+ if (!used[idx])
+ used[idx] = true;
+ }
+
+ /* find the minimal unused index */
+ for (i = 0; i < key->partnparts; i++)
+ if (!used[i])
+ break;
+
+ return i;
+}
+
/* Module-local functions */
/*
@@ -1460,6 +1560,43 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
}
/*
+ * get_qual_for_hash
+ *
+ * Returns a list of expressions to use as a hash partition's constraint.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ List *result;
+ Expr *keyCol;
+ Expr *expr;
+ Expr *opexpr;
+ Oid operoid;
+ ParseState *pstate = make_parsestate(NULL);
+
+ /* Left operand */
+ if (key->partattrs[0] != 0)
+ keyCol = (Expr *) makeVar(1,
+ key->partattrs[0],
+ key->parttypid[0],
+ key->parttypmod[0],
+ key->parttypcoll[0],
+ 0);
+ else
+ keyCol = (Expr *) copyObject(linitial(key->partexprs));
+
+ expr = convert_expr_for_hash(keyCol, key->partnparts, key->parthashfunc);
+
+ /* equals the listdaums value */
+ operoid = LookupOperName(pstate, list_make1(makeString("=")), INT4OID, INT4OID, false, -1);
+ opexpr = make_opclause(operoid, BOOLOID, false, expr, linitial(spec->listdatums), 0, 0);
+
+ result = list_make1(opexpr);
+
+ return result;
+}
+
+/*
* get_partition_operator
*
* Return oid of the operator of given strategy for a given partition key
@@ -1725,6 +1862,11 @@ get_partition_for_tuple(PartitionDispatch *pd,
(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("range partition key of row contains null")));
}
+ else if (key->strategy == PARTITION_STRATEGY_HASH)
+ {
+ values[0] = OidFunctionCall1(key->parthashfunc, values[0]);
+ values[0] = Int16GetDatum(Abs(DatumGetInt16(values[0])) % key->partnparts);
+ }
if (partdesc->boundinfo->has_null && isnull[0])
/* Tuple maps to the null-accepting list partition */
@@ -1739,6 +1881,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
if (cur_offset >= 0 && equal)
cur_index = partdesc->boundinfo->indexes[cur_offset];
else
@@ -1964,6 +2107,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
switch (key->strategy)
{
case PARTITION_STRATEGY_LIST:
+ case PARTITION_STRATEGY_HASH:
cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
bound_datums[0],
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 86329e5..d405cd9 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -41,6 +41,7 @@
#include "catalog/pg_inherits_fn.h"
#include "catalog/pg_namespace.h"
#include "catalog/pg_opclass.h"
+#include "catalog/pg_proc.h"
#include "catalog/pg_tablespace.h"
#include "catalog/pg_trigger.h"
#include "catalog/pg_type.h"
@@ -77,6 +78,7 @@
#include "parser/parse_oper.h"
#include "parser/parse_relation.h"
#include "parser/parse_type.h"
+#include "parser/parse_func.h"
#include "parser/parse_utilcmd.h"
#include "parser/parser.h"
#include "pgstat.h"
@@ -452,7 +454,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid,
Oid oldrelid, void *arg);
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,
+static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes,
List **partexprs, Oid *partopclass, Oid *partcollation);
static void CreateInheritance(Relation child_rel, Relation parent_rel);
static void RemoveInheritance(Relation child_rel, Relation parent_rel);
@@ -801,8 +803,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
AttrNumber partattrs[PARTITION_MAX_KEYS];
Oid partopclass[PARTITION_MAX_KEYS];
Oid partcollation[PARTITION_MAX_KEYS];
+ Oid partatttypes[PARTITION_MAX_KEYS];
List *partexprs = NIL;
List *cmds = NIL;
+ Oid hashfuncOid = InvalidOid;
/*
* We need to transform the raw parsetrees corresponding to partition
@@ -813,15 +817,40 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
stmt->partspec = transformPartitionSpec(rel, stmt->partspec,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
- partattrs, &partexprs, partopclass,
+ partattrs, partatttypes, &partexprs, partopclass,
partcollation);
partnatts = list_length(stmt->partspec->partParams);
+
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ Oid funcrettype;
+
+ if (partnatts != 1)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partition key must be 1 for hash partition")));
+
+ hashfuncOid = LookupFuncName(stmt->partspec->hashfunc, 1, partatttypes, false);
+ funcrettype = get_func_rettype(hashfuncOid);
+ if (funcrettype != INT4OID)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function for partitioning must return integer")));
+
+ if (func_volatile(hashfuncOid) != PROVOLATILE_IMMUTABLE)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function for partitioning must be marked IMMUTABLE")));
+
+ }
+
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
- partopclass, partcollation);
+ partopclass, partcollation, stmt->partspec->partnparts, hashfuncOid);
- /* Force key columns to be NOT NULL when using range partitioning */
- if (strategy == PARTITION_STRATEGY_RANGE)
+ /* Force key columns to be NOT NULL when using range or hash partitioning */
+ if (strategy == PARTITION_STRATEGY_RANGE ||
+ strategy == PARTITION_STRATEGY_HASH)
{
for (i = 0; i < partnatts; i++)
{
@@ -12804,18 +12833,51 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
newspec->strategy = partspec->strategy;
newspec->location = partspec->location;
newspec->partParams = NIL;
+ newspec->partnparts = partspec->partnparts;
+ newspec->hashfunc = partspec->hashfunc;
/* Parse partitioning strategy name */
if (!pg_strcasecmp(partspec->strategy, "list"))
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("unrecognized partitioning strategy \"%s\"",
partspec->strategy)));
+ if (*strategy == PARTITION_STRATEGY_HASH)
+ {
+ if (partspec->partnparts < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partitions must be specified for hash partition")));
+ else if (partspec->partnparts == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partitions must be greater than 0")));
+
+ if (list_length(partspec->hashfunc) == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function must be specified for hash partition")));
+ }
+ else
+ {
+ if (partspec->partnparts >= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("number of partitions can be specified only for hash partition")));
+
+ if (list_length(partspec->hashfunc) > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("hash function can be specified only for hash partition")));
+ }
+
/*
* Create a dummy ParseState and insert the target relation as its sole
* rangetable entry. We need a ParseState for transformExpr.
@@ -12864,7 +12926,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
* Compute per-partition-column information from a list of PartitionElem's
*/
static void
-ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
+ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs, Oid *partatttypes,
List **partexprs, Oid *partopclass, Oid *partcollation)
{
int attn;
@@ -13031,6 +13093,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
"btree",
BTREE_AM_OID);
+ partatttypes[attn] = atttype;
attn++;
}
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 25fd051..9fd27e6 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4329,6 +4329,8 @@ _copyPartitionSpec(const PartitionSpec *from)
COPY_STRING_FIELD(strategy);
COPY_NODE_FIELD(partParams);
+ COPY_SCALAR_FIELD(partnparts);
+ COPY_NODE_FIELD(hashfunc);
COPY_LOCATION_FIELD(location);
return newnode;
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 67529e3..cb57f7b 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2767,6 +2767,8 @@ _equalPartitionSpec(const PartitionSpec *a, const PartitionSpec *b)
{
COMPARE_STRING_FIELD(strategy);
COMPARE_NODE_FIELD(partParams);
+ COMPARE_SCALAR_FIELD(partnparts);
+ COMPARE_NODE_FIELD(hashfunc);
COMPARE_LOCATION_FIELD(location);
return true;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 7418fbe..33305ed 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3421,6 +3421,8 @@ _outPartitionSpec(StringInfo str, const PartitionSpec *node)
WRITE_STRING_FIELD(strategy);
WRITE_NODE_FIELD(partParams);
+ WRITE_INT_FIELD(partnparts);
+ WRITE_NODE_FIELD(hashfunc);
WRITE_LOCATION_FIELD(location);
}
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index 6316688..5054955 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -580,6 +580,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
%type <list> range_datum_list
+%type <ival> hash_partitions
+%type <list> hash_function
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -657,7 +659,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
OBJECT_P OF OFF OFFSET OIDS OLD ON ONLY OPERATOR OPTION OPTIONS OR
ORDER ORDINALITY OUT_P OUTER_P OVER OVERLAPS OVERLAY OWNED OWNER
- PARALLEL PARSER PARTIAL PARTITION PASSING PASSWORD PLACING PLANS POLICY
+ PARALLEL PARSER PARTIAL PARTITION PARTITIONS PASSING PASSWORD PLACING PLANS POLICY
POSITION PRECEDING PRECISION PRESERVE PREPARE PREPARED PRIMARY
PRIOR PRIVILEGES PROCEDURAL PROCEDURE PROGRAM PUBLICATION
@@ -2588,6 +2590,16 @@ ForValues:
$$ = (Node *) n;
}
+
+ /* a HASH partition */
+ | /*EMPTY*/
+ {
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+
+ $$ = (Node *) n;
+ }
;
partbound_datum:
@@ -3667,7 +3679,7 @@ OptPartitionSpec: PartitionSpec { $$ = $1; }
| /*EMPTY*/ { $$ = NULL; }
;
-PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
+PartitionSpec: PARTITION BY part_strategy '(' part_params ')' hash_partitions hash_function
{
PartitionSpec *n = makeNode(PartitionSpec);
@@ -3675,10 +3687,21 @@ PartitionSpec: PARTITION BY part_strategy '(' part_params ')'
n->partParams = $5;
n->location = @1;
+ n->partnparts = $7;
+ n->hashfunc = $8;
+
$$ = n;
}
;
+hash_partitions: PARTITIONS Iconst { $$ = $2; }
+ | /*EMPTY*/ { $$ = -1; }
+ ;
+
+hash_function: USING handler_name { $$ = $2; }
+ | /*EMPTY*/ { $$ = NULL; }
+ ;
+
part_strategy: IDENT { $$ = $1; }
| unreserved_keyword { $$ = pstrdup($1); }
;
@@ -14545,6 +14568,7 @@ unreserved_keyword:
| PARSER
| PARTIAL
| PARTITION
+ | PARTITIONS
| PASSING
| PASSWORD
| PLANS
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 673276a..86f2162 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -40,6 +40,7 @@
#include "catalog/pg_opclass.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_type.h"
+#include "catalog/partition.h"
#include "commands/comment.h"
#include "commands/defrem.h"
#include "commands/tablecmds.h"
@@ -3250,6 +3251,24 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
++i;
}
}
+ else if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ Value *conval;
+ Node *value;
+ int index;
+
+ if (spec->strategy != PARTITION_STRATEGY_HASH)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition")));
+
+ index = get_next_hash_partition_index(parent);
+
+ /* store the partition index as a listdatums value */
+ conval = makeInteger(index);
+ value = (Node *) make_const(pstate, conval, -1);
+ result_spec->listdatums = list_make1(value);
+ }
else
elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 5c82325..9cf79fe 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1426,7 +1426,7 @@ pg_get_indexdef_worker(Oid indexrelid, int colno,
*
* 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)
@@ -1530,6 +1530,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags,
if (!attrsOnly)
appendStringInfo(&buf, "RANGE");
break;
+ case PARTITION_STRATEGY_HASH:
+ appendStringInfo(&buf, "HASH");
+ break;
default:
elog(ERROR, "unexpected partition strategy: %d",
(int) form->partstrat);
@@ -8545,6 +8548,9 @@ get_rule_expr(Node *node, deparse_context *context,
appendStringInfoString(buf, ")");
break;
+ case PARTITION_STRATEGY_HASH:
+ break;
+
default:
elog(ERROR, "unrecognized partition strategy: %d",
(int) spec->strategy);
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index ce55fc5..8455a78 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -855,6 +855,9 @@ RelationBuildPartitionKey(Relation relation)
key->strategy = form->partstrat;
key->partnatts = form->partnatts;
+ key->partnparts = form->partnparts;
+ key->parthashfunc = form->parthashfunc;
+
/*
* We can rely on the first variable-length attribute being mapped to the
* relevant field of the catalog's C struct, because all previous
@@ -999,6 +1002,9 @@ copy_partition_key(PartitionKey fromkey)
newkey->strategy = fromkey->strategy;
newkey->partnatts = n = fromkey->partnatts;
+ newkey->partnparts = fromkey->partnparts;
+ newkey->parthashfunc = fromkey->parthashfunc;
+
newkey->partattrs = (AttrNumber *) palloc(n * sizeof(AttrNumber));
memcpy(newkey->partattrs, fromkey->partattrs, n * sizeof(AttrNumber));
diff --git a/src/include/catalog/heap.h b/src/include/catalog/heap.h
index 1187797..367e2f8 100644
--- a/src/include/catalog/heap.h
+++ b/src/include/catalog/heap.h
@@ -141,7 +141,7 @@ extern void StorePartitionKey(Relation rel,
AttrNumber *partattrs,
List *partexprs,
Oid *partopclass,
- Oid *partcollation);
+ Oid *partcollation, int16 partnparts, Oid hashfunc);
extern void RemovePartitionKeyByRelId(Oid relid);
extern void StorePartitionBound(Relation rel, Relation parent, Node *bound);
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 421644c..b1102f3 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -95,4 +95,6 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
EState *estate,
PartitionDispatchData **failed_at,
TupleTableSlot **failed_slot);
+extern Expr *convert_expr_for_hash(Expr *expr, int npart, Oid hashfunc);
+extern int get_next_hash_partition_index(Relation parent);
#endif /* PARTITION_H */
diff --git a/src/include/catalog/pg_partitioned_table.h b/src/include/catalog/pg_partitioned_table.h
index bdff36a..de526a5 100644
--- a/src/include/catalog/pg_partitioned_table.h
+++ b/src/include/catalog/pg_partitioned_table.h
@@ -33,6 +33,9 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
char partstrat; /* partitioning strategy */
int16 partnatts; /* number of partition key columns */
+ int16 partnparts; /* number of partitions for hash partitioning */
+ Oid parthashfunc; /* hash function oid for hash partitioning */
+
/*
* variable-length fields start here, but we allow direct access to
* partattrs via the C struct. That's because the first variable-length
@@ -62,13 +65,15 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table;
* compiler constants for pg_partitioned_table
* ----------------
*/
-#define Natts_pg_partitioned_table 7
+#define Natts_pg_partitioned_table 9
#define Anum_pg_partitioned_table_partrelid 1
#define Anum_pg_partitioned_table_partstrat 2
#define Anum_pg_partitioned_table_partnatts 3
-#define Anum_pg_partitioned_table_partattrs 4
-#define Anum_pg_partitioned_table_partclass 5
-#define Anum_pg_partitioned_table_partcollation 6
-#define Anum_pg_partitioned_table_partexprs 7
+#define Anum_pg_partitioned_table_partnparts 4
+#define Anum_pg_partitioned_table_parthashfunc 5
+#define Anum_pg_partitioned_table_partattrs 6
+#define Anum_pg_partitioned_table_partclass 7
+#define Anum_pg_partitioned_table_partcollation 8
+#define Anum_pg_partitioned_table_partexprs 9
#endif /* PG_PARTITIONED_TABLE_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index d576523..6db2301 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -761,13 +761,16 @@ typedef struct PartitionElem
typedef struct PartitionSpec
{
NodeTag type;
- char *strategy; /* partitioning strategy ('list' or 'range') */
+ char *strategy; /* partitioning strategy ('list', 'range', or 'hash') */
List *partParams; /* List of PartitionElems */
+ int partnparts; /* number of partitions for hash partitioning */
+ List *hashfunc; /* hash function for hash partitioning */
int location; /* token location, or -1 if unknown */
} PartitionSpec;
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
+#define PARTITION_STRATEGY_HASH 'h'
/*
* PartitionBoundSpec - a partition bound specification
diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h
index 28c4dab..b8d898e 100644
--- a/src/include/parser/kwlist.h
+++ b/src/include/parser/kwlist.h
@@ -292,6 +292,7 @@ PG_KEYWORD("parallel", PARALLEL, UNRESERVED_KEYWORD)
PG_KEYWORD("parser", PARSER, UNRESERVED_KEYWORD)
PG_KEYWORD("partial", PARTIAL, UNRESERVED_KEYWORD)
PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD)
+PG_KEYWORD("partitions", PARTITIONS, UNRESERVED_KEYWORD)
PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD)
PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD)
PG_KEYWORD("placing", PLACING, RESERVED_KEYWORD)
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index a617a7c..660adfb 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -62,6 +62,9 @@ typedef struct PartitionKeyData
Oid *partopcintype; /* OIDs of opclass declared input data types */
FmgrInfo *partsupfunc; /* lookup info for support funcs */
+ int16 partnparts; /* number of hash partitions */
+ Oid parthashfunc; /* OID of hash function */
+
/* Partitioning collation per attribute */
Oid *partcollation;
Please post an explanation for the delay and a schedule for the new
patch. If no patch or explanation is posted by 2017-03-17 AoE I will
mark this submission "Returned with Feedback".
Depite the fact that Yugo has posted a new patch on 2017-03-17, this
item had been marked as "Returned with Feedback". I don't know why.
--
Tatsuo Ishii
SRA OSS, Inc. Japan
English: http://www.sraoss.co.jp/index_en.php
Japanese:http://www.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
On Fri, Mar 17, 2017 at 7:57 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
I also understanded that my design has a problem during pg_dump and
pg_upgrade, and that some information to identify the partition
is required not depending the command order. However, I feel that
Amul's design is a bit complicated with the rule to specify modulus.I think we can use simpler syntax, for example, as below.
CREATE TABLE h1 PARTITION OF h FOR (0);
CREATE TABLE h2 PARTITION OF h FOR (1);
CREATE TABLE h3 PARTITION OF h FOR (2);
I don't see how that can possibly work. Until you see all the table
partitions, you don't know what the partitioning constraint for any
given partition should be, which seems to me to be a fatal problem.
I agree that Amul's syntax - really, I proposed it to him - is not the
simplest, but I think all the details needed to reconstruct the
partitioning constraint need to be explicit. Otherwise, I'm pretty
sure things we're going to have lots of problems that we can't really
solve cleanly. We can later invent convenience syntax that makes
common configurations easier to set up, but we should invent the
syntax that spells out all the details first.
--
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, 13 Apr 2017 16:40:29 -0400
Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Mar 17, 2017 at 7:57 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
I also understanded that my design has a problem during pg_dump and
pg_upgrade, and that some information to identify the partition
is required not depending the command order. However, I feel that
Amul's design is a bit complicated with the rule to specify modulus.I think we can use simpler syntax, for example, as below.
CREATE TABLE h1 PARTITION OF h FOR (0);
CREATE TABLE h2 PARTITION OF h FOR (1);
CREATE TABLE h3 PARTITION OF h FOR (2);I don't see how that can possibly work. Until you see all the table
partitions, you don't know what the partitioning constraint for any
given partition should be, which seems to me to be a fatal problem.
If a partition has an id, the partitioning constraint can be written as
hash_func(hash_key) % N = id
wehre N is the number of paritions. Doesn't it work?
I agree that Amul's syntax - really, I proposed it to him - is not the
simplest, but I think all the details needed to reconstruct the
partitioning constraint need to be explicit. Otherwise, I'm pretty
sure things we're going to have lots of problems that we can't really
solve cleanly. We can later invent convenience syntax that makes
common configurations easier to set up, but we should invent the
syntax that spells out all the details first.
I have a question about Amul's syntax. After we create partitions
as followings,
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);
we cannot create any additional partitions for the partition.
Then, after inserting records into foo1 and foo2, how we can
increase the number of partitions?
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
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
On Fri, Apr 14, 2017 at 4:23 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
On Thu, 13 Apr 2017 16:40:29 -0400
Robert Haas <robertmhaas@gmail.com> wrote:On Fri, Mar 17, 2017 at 7:57 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
I also understanded that my design has a problem during pg_dump and
pg_upgrade, and that some information to identify the partition
is required not depending the command order. However, I feel that
Amul's design is a bit complicated with the rule to specify modulus.I think we can use simpler syntax, for example, as below.
CREATE TABLE h1 PARTITION OF h FOR (0);
CREATE TABLE h2 PARTITION OF h FOR (1);
CREATE TABLE h3 PARTITION OF h FOR (2);I don't see how that can possibly work. Until you see all the table
partitions, you don't know what the partitioning constraint for any
given partition should be, which seems to me to be a fatal problem.If a partition has an id, the partitioning constraint can be written as
hash_func(hash_key) % N = id
wehre N is the number of paritions. Doesn't it work?
Only if you know the number of partitions. But with your syntax,
after seeing only the first of the CREATE TABLE .. PARTITION OF
commands, what should the partition constraint be? It depends on how
many more such commands appear later in the dump file, which you do
not know at that point.
I agree that Amul's syntax - really, I proposed it to him - is not the
simplest, but I think all the details needed to reconstruct the
partitioning constraint need to be explicit. Otherwise, I'm pretty
sure things we're going to have lots of problems that we can't really
solve cleanly. We can later invent convenience syntax that makes
common configurations easier to set up, but we should invent the
syntax that spells out all the details first.I have a question about Amul's syntax. After we create partitions
as followings,create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);we cannot create any additional partitions for the partition.
Then, after inserting records into foo1 and foo2, how we can
increase the number of partitions?
You can detach foo1, create two new partitions with modulus 4 and
remainders 0 and 2, and move the data over from the old partition.
I realize that's not as automated as you might like, but it's no worse
than what is currently required for list and range partitioning when
you split a partition. Someday we might build in tools to do that
kind of data migration automatically, but right now we have none.
--
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, 14 Apr 2017 09:05:14 -0400
Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Apr 14, 2017 at 4:23 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
On Thu, 13 Apr 2017 16:40:29 -0400
Robert Haas <robertmhaas@gmail.com> wrote:On Fri, Mar 17, 2017 at 7:57 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
I also understanded that my design has a problem during pg_dump and
pg_upgrade, and that some information to identify the partition
is required not depending the command order. However, I feel that
Amul's design is a bit complicated with the rule to specify modulus.I think we can use simpler syntax, for example, as below.
CREATE TABLE h1 PARTITION OF h FOR (0);
CREATE TABLE h2 PARTITION OF h FOR (1);
CREATE TABLE h3 PARTITION OF h FOR (2);I don't see how that can possibly work. Until you see all the table
partitions, you don't know what the partitioning constraint for any
given partition should be, which seems to me to be a fatal problem.If a partition has an id, the partitioning constraint can be written as
hash_func(hash_key) % N = id
wehre N is the number of paritions. Doesn't it work?
Only if you know the number of partitions. But with your syntax,
after seeing only the first of the CREATE TABLE .. PARTITION OF
commands, what should the partition constraint be? It depends on how
many more such commands appear later in the dump file, which you do
not know at that point.
I thought that the partition constraint could be decided every
time a new partition is created or attached, and that it woule be
needed to relocate records automatically when the partition configuration
changes. However, I have come to think that the automatic relocation
might not be needed at this point.
I agree that Amul's syntax - really, I proposed it to him - is not the
simplest, but I think all the details needed to reconstruct the
partitioning constraint need to be explicit. Otherwise, I'm pretty
sure things we're going to have lots of problems that we can't really
solve cleanly. We can later invent convenience syntax that makes
common configurations easier to set up, but we should invent the
syntax that spells out all the details first.I have a question about Amul's syntax. After we create partitions
as followings,create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);we cannot create any additional partitions for the partition.
Then, after inserting records into foo1 and foo2, how we can
increase the number of partitions?You can detach foo1, create two new partitions with modulus 4 and
remainders 0 and 2, and move the data over from the old partition.I realize that's not as automated as you might like, but it's no worse
than what is currently required for list and range partitioning when
you split a partition. Someday we might build in tools to do that
kind of data migration automatically, but right now we have none.
Thanks. I understood it. The automatic data migration feature
would be better to be implemented separately.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
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
On Mon, Apr 17, 2017 at 10:50 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
I thought that the partition constraint could be decided every
time a new partition is created or attached, and that it woule be
needed to relocate records automatically when the partition configuration
changes. However, I have come to think that the automatic relocation
might not be needed at this point.
Great! I am glad that we are in agreement about this point. However,
actually I think the problem is worse than you are supposing. If
you're restoring from a database dump created by pg_dump, then we will
try to load data into each individual partition using COPY. Direct
insertions into individual partitions are not subject to tuple routing
-- that only affects inserts into the parent table. So if the
partition constraint is not correct immediately after creating the
table, the COPY which tries to repopulate that partition will probably
fail with an ERROR, because there will likely be at least one row
(probably many) which match the "final" partition constraint but not
the "interim" partition constraint that we'd have after recreating
some but not all of the hash partitions. For example, if we had
created 2 partitions so far out of a total of 3, we'd think the
constraint ought to be (hashvalue % 2) == 1 rather than (hashvalue %
3) == 1, which obviously will likely lead to the dump failing to
restore properly.
So, I think we really need something like the syntax in Amul's patch
in order for this to work at all. Of course, the details can be
changed according to what seems best but I think the overall picture
is about right.
There is another point that I think also needs thought; not sure if
either your patch or Amit's patch handles it: constraint exclusion
will not work for hash partitioning. For example, if the partitioning
constraint for each partition is of the form (hash(partcol) % 6) ==
SOME_VALUE_BETWEEN_0_AND_5, and the query contains the predicate
partcol == 37, constraint exclusion will not be able to prove anything
about which partitions need to be scanned. Amit Langote has noted a
few times that partitioning relies on constraint exclusion *for now*,
which implies, I think, that he's thought about changing it to work
differently. I think that would be a good idea. For range
partitioning or list partitioning, a special-purpose mechanism for
partitioning could be much faster than constraint exclusion, since it
knows that partcol == 37 can only be true for one partition and can
reuse the tuple-routing infrastructure to figure out which one it is.
And that approach can also work for hash partitioning, where
constraint exclusion is useless.
--
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, Apr 20, 2017 at 4:27 PM, Robert Haas <robertmhaas@gmail.com> wrote:
So, I think we really need something like the syntax in Amul's patch
in order for this to work at all. Of course, the details can be
changed according to what seems best but I think the overall picture
is about right.
I spent some time today looking at these patches. It seems like there
is some more work still needed here to produce something committable
regardless of which way we go, but I am inclined to think that Amul's
patch is a better basis for work going forward than Nagata-san's
patch. Here are some general comments on the two patches:
- As noted above, the syntax implemented by Amul's patch allows us to
know the final partition constraint right away. Nagata-san's proposed
syntax does not do that. Also, Amul's syntax allows for a way to
split partitions (awkwardly, but we can improve it later);
Nagata-san's doesn't provide any method at all.
- Amul's patch derives the hash function to be used from the relevant
hash opclass, whereas Nagata-san's patch requires the user to specify
it explicitly. I think that there is no real use case for a user
providing a custom hash function, and that using the opclass machinery
to derive the function to be used is better. If a user DOES want to
provide their own, they can always create a custom opclass with the
appropriate support function and specify that it should be used when
creating a hash-partitioned table, but most users will be happy for
the system to supply the appropriate function automatically.
- In Nagata-san's patch, convert_expr_for_hash() looks up a function
called "abs" and an operator called "%" by name, which is not a good
idea. We don't want to just find whatever is in the current search
path; we want to make sure we're using the system-defined operators
that we intend to be using. Amul's patch builds the constraint using
a hard-coded internal function OID, F_SATISFIES_HASH_PARTITION.
That's a lot more robust, and it's also likely to be faster because,
in Amul's patch, we only call one function at the SQL level
(satisfies_hash_partition), whereas in Nagata-san's patch, we'll end
up calling three (abs, %, =). Nagata-san's version of
get_qual_for_hash is implicated in this problem, too: it's looking up
the operator to use based on the operator name (=) rather than the
opclass properties. Note that the existing get_qual_for_list() and
get_qual_for_range() use opclass properties, as does Amul's patch.
- Nagata-san's patch only supports hash partitioning based on a single
column, and that column must be NOT NULL. Amul's patch does not have
these restrictions.
- Neither patch contains any documentation updates, which is bad.
Nagata-san's patch also contains no regression tests. Amul's patch
does, but they need to be rebased, since they no longer apply, and I
think some other improvements are possible as well. It's probably not
necessary to re-test things like whether temp and non-temp tables can
be mixed within a partitioning hierarchy, but there should be tests
that tuple routing actually works. The case where it fails because no
matching partition exists should be tested as well. Also, the tests
should validate not only that FOR VALUES isn't accept when creating a
hash partition (which they do) but also that WITH (...) isn't accepted
for a range or list partition (which they do not).
- When I try to do even something pretty trivial with Nagata-san's
patches, it crashes:
rhaas=# create table foo (a int, b text) partition by hash (a)
partitions 7 using hashint4;
CREATE TABLE
rhaas=# create table foo1 partition of foo;
<server crash>
The ruleutils.c support in Nagata-san's patch is broken. If you
execute the non-crashing statement from the above example and then run
pg_dump, it doesn't dump "partitions 7 using hashint4", which means
that the syntax in the dump is invalid.
- Neither patch does anything about the fact that constraint exclusion
won't work for hash partitioning. I mentioned this issue upthread in
the last paragraph of
/messages/by-id/CA+Tgmob7RsN5A=ehgYbLPx--c5CmptrK-dB=Y-v--o+TKyfteA@mail.gmail.com
and I think it's imperative that we fix it in some way before we think
about committing any of this. I think that needs to be done by
extending relation_excluded_by_constraints() to have some specific
smarts about hash partitioning, and maybe other kinds of partitioning
as well (because it could probably be made much faster for list and
range partitioning, too).
- Amul's patch should perhaps update tab completion support: create
table foo1 partition of foo <tab> completes with "for values", but now
"with" will be another option.
- Amul's patch probably needs to validate the WITH () clause more
thoroughly. I bet you get a not-very-great error message if you leave
out "modulus" and no error at all if you leave out "remainder".
This is not yet a detailed review - I may be missing things, and
review and commentary from others is welcome. If there is no major
disagreement with the idea of moving forward using Amul's patch as a
base, then I will do a more detailed review of that patch (or,
hopefully, an updated version that addresses the above comments).
--
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, Feb 28, 2017 at 6:33 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.
I suggest that we consider the hash functions more carefully. This is
(effectively) an on-disk format so it can't be changed easily later.
1. Consider a partition-wise join of two hash-partitioned tables. If
that's a hash join, and we just use the hash opclass, we immediately
lose some useful bits of the hash function. Same for hash aggregation
where the grouping key is the partition key.
To fix this, I think we need to include a salt in the hash API. Each
level of hashing can choose a random salt.
2. Consider a partition-wise join where the join keys are varchar(10)
and char(10). We can't do that join if we just use the existing hash
strategy, because 'foo' = 'foo ' should match, but those values
have different hashes when using the standard hash opclass.
To fix this, we need to be smarter about normalizing values at a
logical level before hashing. We can take this to varying degrees,
perhaps even normalizing an integer to a numeric before hashing so
that you can do a cross-type join on int=numeric.
Furthermore, we need catalog metadata to indicate which hash functions
are suitable for which cross-type comparisons. Or, to put it the other
way, which typecasts preserve the partitioning.
3. We might want to use a hash function that is a little slower that
is more resistant to collisions. We may even want to use a 64-bit
hash.
My opinion is that we should work on this hashing infrastructure
first, and then support the DDL. If we get the hash functions right,
that frees us up to create better plans, with better push-downs, which
will be good for parallel query.
Regards,
Jeff Davis
--
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, May 2, 2017 at 9:01 PM, Jeff Davis <pgsql@j-davis.com> wrote:
On Tue, Feb 28, 2017 at 6:33 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
In this patch, user can specify a hash function USING. However,
we migth need default hash functions which are useful and
proper for hash partitioning.I suggest that we consider the hash functions more carefully. This is
(effectively) an on-disk format so it can't be changed easily later.1. Consider a partition-wise join of two hash-partitioned tables. If
that's a hash join, and we just use the hash opclass, we immediately
lose some useful bits of the hash function. Same for hash aggregation
where the grouping key is the partition key.
Hmm, that could be a problem in some cases. I think there's probably
much less of a problem if the modulus isn't a power of two?
To fix this, I think we need to include a salt in the hash API. Each
level of hashing can choose a random salt.
Do you mean that we'd salt partitioning hashing differently from
grouping hashing which would be salted different from aggregation
hashing which, I suppose, would be salted differently from hash index
hashing? Or do you mean that you'd have to specify a salt when
creating a hash-partitioned table, and make sure it's the same across
all compatibly partitioned tables you might want to hash-join? That
latter sounds unappealing.
2. Consider a partition-wise join where the join keys are varchar(10)
and char(10). We can't do that join if we just use the existing hash
strategy, because 'foo' = 'foo ' should match, but those values
have different hashes when using the standard hash opclass.To fix this, we need to be smarter about normalizing values at a
logical level before hashing. We can take this to varying degrees,
perhaps even normalizing an integer to a numeric before hashing so
that you can do a cross-type join on int=numeric.Furthermore, we need catalog metadata to indicate which hash functions
are suitable for which cross-type comparisons. Or, to put it the other
way, which typecasts preserve the partitioning.
You're basically describing what a hash opfamily already does, except
that we don't have a single opfamily that covers both varchar(10) and
char(10), nor do we have one that covers both int and numeric. We
have one that covers int2, int4, and int8, though. If somebody wanted
to make the ones you're suggesting, there's nothing preventing it,
although I'm not sure exactly how we'd encourage people to start using
the new one and deprecating the old one. We don't seem to have a good
infrastructure for that.
3. We might want to use a hash function that is a little slower that
is more resistant to collisions. We may even want to use a 64-bit
hash.My opinion is that we should work on this hashing infrastructure
first, and then support the DDL. If we get the hash functions right,
that frees us up to create better plans, with better push-downs, which
will be good for parallel query.
I am opposed to linking the fate of this patch to multiple
independent, possibly large, possibly difficult, possibly
controversial enhancements to the hashing mechanism. If there are
simple things that can reasonably be done in this patch to make hash
partitioning better, great. If you want to work on improving the
hashing mechanism as an independent project, also great. But I think
that most people would rather have hash partitioning in v11 than wait
for v12 or v13 so that other hashing improvements can be completed; I
know I would. If we say "we shouldn't implement hash partitioning
because some day we might make incompatible changes to the hashing
mechanism" then we'll never implement it, because that will always be
true. Even the day after we change it, there still may come a future
day when we change it again.
The stakes have already been raised by making hash indexes durable;
that too is arguably making future changes to the hashing
infrastructure harder. But I still think it was the right thing to
proceed with that work. If we get 64-bit hash codes in the next
release, and we want hash indexes to use them, then we will have to
invalidate existing hash indexes (again). That's sad, but not as sad
as it would have been to not commit the work to make hash indexes
durable. There's a chicken-and-egg problem here: without durable hash
indexes and hash partitioning, there's not much incentive to make
hashing better, but once we have them, changes create a backward
compatibility issue. Such is life; nothing we do is infinitely
future-proof.
The last significant overhaul of the hashing mechanism that I know
about was in 2009, cf. 2604359251d34177a14ef58250d8b4a51d83103b and
8205258fa675115439017b626c4932d5fefe2ea8. Until this email, I haven't
seen any complaints about the quality of that hash function either in
terms of speed or collision properties - what makes you think those
things are serious problems? I *have* heard some interest in widening
the output to 64 bits, and also in finding a way to combine multiple
hash values in some smarter way than we do at present. Seeding has
come up, too.
--
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, Apr 27, 2017 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:
I spent some time today looking at these patches. It seems like there
is some more work still needed here to produce something committable
regardless of which way we go, but I am inclined to think that Amul's
patch is a better basis for work going forward than Nagata-san's
patch. Here are some general comments on the two patches:
Thanks for your time.
[...]
- Neither patch contains any documentation updates, which is bad.
Fixed in the attached version.
Nagata-san's patch also contains no regression tests. Amul's patch
does, but they need to be rebased, since they no longer apply, and I
think some other improvements are possible as well. It's probably not
necessary to re-test things like whether temp and non-temp tables can
be mixed within a partitioning hierarchy, but there should be tests
that tuple routing actually works. The case where it fails because no
matching partition exists should be tested as well. Also, the tests
should validate not only that FOR VALUES isn't accept when creating a
hash partition (which they do) but also that WITH (...) isn't accepted
for a range or list partition (which they do not).
Fixed in the attached version.
[...]
- Amul's patch should perhaps update tab completion support: create
table foo1 partition of foo <tab> completes with "for values", but now
"with" will be another option.
Fixed in the attached version.
- Amul's patch probably needs to validate the WITH () clause more
thoroughly. I bet you get a not-very-great error message if you leave
out "modulus" and no error at all if you leave out "remainder".
Thats not true, there will be syntax error if you leave modulus or
remainder, see this:
postgres=# CREATE TABLE hpart_2 PARTITION OF hash_parted WITH(modulus 4);
ERROR: syntax error at or near ")"
LINE 1: ...hpart_2 PARTITION OF hash_parted WITH(modulus 4);
This is not yet a detailed review - I may be missing things, and
review and commentary from others is welcome. If there is no major
disagreement with the idea of moving forward using Amul's patch as a
base, then I will do a more detailed review of that patch (or,
hopefully, an updated version that addresses the above comments).
I have made a smaller change in earlier proposed syntax to create
partition to be aligned with current range and list partition syntax,
new syntax will be as follow:
CREATE TABLE p1 PARTITION OF hash_parted FOR VALUES WITH (modulus 10,
remainder 1);
Regards,
Amul
Attachments:
hash-partitioning_another_design-v2.patchapplication/octet-stream; name=hash-partitioning_another_design-v2.patchDownload
From b6cf0ef3fe1c0c95277e7b183edc3afb3bea7b4f Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Wed, 3 May 2017 18:28:00 +0530
Subject: [PATCH] Declarative hash partitioning v2
v2:
Regression test updated.
Documentation added.
Added tab complition for FOR VALUES WITH
v1:
Initial patch
---
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 52 ++-
src/backend/catalog/partition.c | 573 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 63 +++-
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 | 47 ++-
src/backend/parser/parse_utilcmd.c | 28 +-
src/backend/utils/adt/ruleutils.c | 11 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 61 +++
src/test/regress/expected/create_table.out | 66 +++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 +++
src/test/regress/sql/create_table.sql | 43 ++-
src/test/regress/sql/insert.sql | 39 ++
src/test/regress/sql/update.sql | 19 +
22 files changed, 1064 insertions(+), 93 deletions(-)
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..3b5b5f9 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 postal_code
+ ATTACH PARTITION postal_code_p2 FOR VALUES WITH (modulus 256, remainder 1);
+</programlisting></para>
+
+ <para>
Detach a partition from partitioned table:
<programlisting>
ALTER TABLE cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..6c9f87a 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 { HASH | RANGE | LIST } ( { <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 { HASH | RANGE | LIST } ( { <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 { HASH | RANGE | LIST } ( { <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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">value</replaceable>, REMAINDER <replaceable class="PARAMETER">value</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,28 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</para>
<para>
+ When creating a hash partition, <literal>MODULUS</literal> should be
+ greater than zero and <literal>REMAINDER</literal> should be greater than
+ or equal to zero. Every <literal>MODULUS</literal> must be a factor of
+ the next larger modulus. For example, if you have a bunch of partitions
+ that all have modulus <literal>5</literal>, you can add a new new
+ partition with modulus <literal>10</literal> or a new partition with
+ modulus <literal>15</literal>, but you cannot add both a partition with
+ modulus <literal>10</literal> and a partition with modulus
+ <literal>15</literal>, because <literal>10</literal> is not a factor of
+ <literal>15</literal>. However, you could simultaneously use modulus
+ <literal>4</literal>, modulus <literal>8</literal>, modulus
+ <literal>16</literal>, and modulus <literal>32</literal> if you wished,
+ because each modulus is a factor of the next larger one. You could also
+ use modulus <literal>10</literal>, modulus <literal>20</literal>, and
+ modulus <literal>60</literal>. But you could not use modulus
+ <literal>10</literal>, modulus <literal>15</literal>, and modulus
+ <literal>60</literal> because while both of the smaller moduli are factors
+ of <literal>60</literal>, it is not true that each is a factor of the
+ next.
+ </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
@@ -422,7 +445,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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 { HASH | RANGE | LIST } ( { <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
@@ -1591,6 +1614,16 @@ CREATE TABLE cities (
</programlisting></para>
<para>
+ Create a hash partitioned table:
+<programlisting>
+CREATE TABLE postal_code (
+ code int not null,
+ city_id bigint not null,
+ address text
+) PARTITION BY HASH (code);
+</programlisting></para>
+
+ <para>
Create partition of a range partitioned table:
<programlisting>
CREATE TABLE measurement_y2016m07
@@ -1641,6 +1674,15 @@ 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 partition of a hash partitioned table:
+<programlisting>
+CREATE TABLE postal_code_p1
+ PARTITION OF postal_code (
+ CONSTRAINT eight_digit CHECK (code > 9999999)
+) FOR VALUES WITH(MODULUS 256, REMAINDER 0);
+</programlisting></para>
</refsect1>
<refsect1 id="SQL-CREATETABLE-compatibility">
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 8641ae1..0a99433 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
@@ -76,16 +77,18 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
Datum **datums; /* Array of datum-tuples with key->partnatts
* datums each */
RangeDatumContent **content;/* what's contained in each range bound datum?
* (see the above enum); NULL for list
* partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ int *indexes; /* Partition indexes; in case of hash
+ * partitioned table array length will be
+ * value of largest modulus, and for others
+ * one entry per member of the datums array
+ * (plus one if range partitioned table) */
bool has_null; /* Is there a null-accepting partition? false
* for range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
@@ -97,6 +100,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
{
@@ -113,11 +124,13 @@ 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,
void *arg);
+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 Oid get_partition_operator(PartitionKey key, int col,
@@ -140,6 +153,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values,
+ bool *isnull);
+/* SQL-callable function for use in hash partition CHECK constraints */
+PG_FUNCTION_INFO_V1(satisfies_hash_partition);
+
/*
* RelationBuildPartitionDesc
* Form rel's partition descriptor
@@ -163,6 +181,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -229,7 +250,33 @@ 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 = 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;
@@ -456,6 +503,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));
+ memset(boundinfo->indexes, -1,
+ greatest_modulus * sizeof(int));
+
+ for (i = 0; i < nparts; i++)
+ {
+ int mod = hbounds[i]->modulus,
+ place = hbounds[i]->remainder;
+
+ boundinfo->datums[i] = (Datum *) palloc(2 *
+ sizeof(Datum));
+ boundinfo->datums[i][0] = Int32GetDatum(mod);
+ boundinfo->datums[i][1] = Int32GetDatum(place);
+ next_index = hbounds[i]->index;
+
+ while (place < greatest_modulus)
+ {
+ /* overlap? */
+ Assert(boundinfo->indexes[place] == -1);
+ boundinfo->indexes[place] = next_index;
+ place = place + mod;
+ }
+
+ mapping[i] = i;
+ pfree(hbounds[i]);
+ }
+ pfree(hbounds);
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
boundinfo->has_null = found_null;
@@ -609,53 +692,77 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
- for (j = 0; j < key->partnatts; j++)
+ /*
+ * Hash partition bound stores modulus and remainder at
+ * b1->datums[i][0] and b1->datums[i][0] position respectively.
+ */
+ for (i = 0; i < b1->ndatums; i++)
+ if (!(datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+ true, sizeof(int)) &&
+ datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+ true, sizeof(int))))
+ return false;
+
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound, and
+ * different kinds of infinities differ from each other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /* Non-finite bounds are equal without further examination. */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -677,6 +784,92 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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 place;
+
+ /*
+ * 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
+ * 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. However, you could simultaneously
+ * use modulus 4, modulus 8, modulus 16, and modulus 32 if
+ * you wished, because each modulus is a factor of the next
+ * larger one. You could also use modulus 10, modulus 20,
+ * and modulus 60. But you could not use modulus 10,
+ * modulus 15, and modulus 60 for the same reason.
+ */
+ {
+ int offset;
+ bool equal,
+ valid_bound = true;
+ int pmod, /* Previous largest modulus */
+ nmod; /* Next largest modulus */
+
+ /*
+ * 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)
+ {
+ nmod = DatumGetInt32(datums[0][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ else
+ {
+ pmod = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % pmod) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ nmod = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("every modulus must be factor of next largest modulus")));
+ }
+
+ greatest_modulus = DatumGetInt32(datums[ndatums - 1][0]);
+ place = spec->remainder;
+
+ if (place >= greatest_modulus)
+ place = place % greatest_modulus;
+
+ do
+ {
+ if (boundinfo->indexes[place] != -1)
+ {
+ overlap = true;
+ with = boundinfo->indexes[place];
+ break;
+ }
+ place = place + spec->modulus;
+ } while (place < greatest_modulus);
+ }
+
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
Assert(spec->strategy == PARTITION_STRATEGY_LIST);
@@ -899,6 +1092,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1146,6 +1344,97 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
/* Module-local functions */
/*
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder this function
+ * returns an expression Node for the partition table's CHECK constraint.
+ *
+ * For example, given a partition definition such as:
+ * CREATE TABLE simple_hash (pkey int, value char(10))
+ * PARTITION BY HASH (pkey, value);
+ *
+ * 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 a
+ * subexpression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, pkey, value)
+ * for p_p2: satisfies_hash_partition(4, 2, pkey, value)
+ * for p_p3: satisfies_hash_partition(8, 0, pkey, value)
+ * for p_p4: satisfies_hash_partition(8, 4, pkey, value)
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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
+ {
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ args = lappend(args, keyCol);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns a list of expressions to use as a list partition's constraint.
@@ -1738,29 +2027,58 @@ get_partition_for_tuple(PartitionDispatch *pd,
cur_index = partdesc->boundinfo->null_index;
else if (!isnull[0])
{
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
-
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
switch (key->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ {
+ PartitionBoundInfo boundinfo = partdesc->boundinfo;
+ int ndatums = boundinfo->ndatums;
+ Datum datum = boundinfo->datums[ndatums - 1][0];
+ int modulus = DatumGetInt32(datum);
+ uint32 rowHash = cal_hash_value(key->partsupfunc,
+ key->partnatts,
+ values, isnull);
+
+ cur_index = boundinfo->indexes[rowHash % modulus];
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
- if (cur_offset >= 0 && equal)
- cur_index = partdesc->boundinfo->indexes[cur_offset];
- else
- cur_index = -1;
- break;
+ {
+ /* bsearch in partdesc->boundinfo */
+ bool equal = false;
+
+ 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;
+ }
case PARTITION_STRATEGY_RANGE:
+ {
+ bool equal = false;
- /*
- * Offset returned is such that the bound at offset is
- * found to be less or equal with the tuple. So, the bound
- * at offset+1 would be the upper bound.
- */
- cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
- break;
+ /* 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
+ * found to be less or equal with the tuple. So, the bound
+ * at offset+1 would be the upper bound.
+ */
+ cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
+ break;
+ }
default:
elog(ERROR, "unexpected partition strategy: %d",
@@ -1794,6 +2112,27 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * Used when sorting hash bounds across all hash modulus
+ * for hash partitioning
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+ PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+ PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+ int v1 = h1->modulus;
+ int v2 = h2->modulus;
+
+ if (v1 < v2)
+ return -1;
+ if (v1 > v2)
+ return 1;
+ if (v1 == v2 && h1->remainder != h2->remainder)
+ return (h1->remainder > h2->remainder) ? 1 : -1;
+ return 0;
+}
+
+/*
* qsort_partition_list_value_cmp
*
* Compare two list partition bound datums
@@ -1970,6 +2309,25 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
switch (key->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ {
+ PartitionBoundSpec *spec = (PartitionBoundSpec *) probe;
+ int mod = DatumGetInt32(bound_datums[0]);
+
+ if (mod < spec->modulus)
+ cmpval = -1;
+ else if (mod > spec->modulus)
+ cmpval = 1;
+ else if (mod == spec->modulus)
+ {
+ int rem = DatumGetInt32(bound_datums[1]);
+
+ cmpval = rem == spec->remainder ? 0 :
+ (rem < spec->remainder? -1 : 1);
+ }
+
+ break;
+ }
case PARTITION_STRATEGY_LIST:
cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
@@ -2053,3 +2411,110 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * Compute the hash value for given partition column values.
+ */
+static uint32
+cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /* rotate hash left 1 bit before mixing in the next column */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ {
+ Datum colHash;
+
+ Assert(OidIsValid(partsupfunc[i].fn_oid));
+
+ colHash = FunctionCall1(&partsupfunc[i], values[i]);
+ rowHash ^= DatumGetUInt32(colHash);
+ }
+ }
+
+ return rowHash;
+}
+
+/*
+ * satisfies_hash_partition
+ *
+ * This is a SQL-callable function for use in hash partition constraints;
+ * see get_qual_for_hash() for usage.
+ */
+Datum
+satisfies_hash_partition(PG_FUNCTION_ARGS)
+{
+ typedef struct ColumnsHashData
+ {
+ int n; /* allocated length of typentry[] */
+ TypeCacheEntry *typentry[PARTITION_MAX_KEYS];
+ } ColumnsHashData;
+ int modulus = PG_GETARG_INT32(0);
+ int remainder = PG_GETARG_INT32(1);
+ short nkeys = PG_NARGS() - 2;
+ int i;
+ Datum values[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ FmgrInfo partsupfunc[PARTITION_MAX_KEYS];
+ ColumnsHashData *my_extra;
+ uint32 rowHash = 0;
+
+ /*
+ * Cache hash function information, similar to how record_eq() caches
+ * equality operator information. (Perhaps no SQL syntax could cause
+ * PG_NARGS()/nkeys to change between calls through the same FmgrInfo.
+ * Checking nkeys here is just defensiveness.)
+ */
+ my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+ if (my_extra == NULL || my_extra->n != nkeys)
+ {
+ fcinfo->flinfo->fn_extra =
+ MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
+ offsetof(ColumnsHashData, typentry) +
+ sizeof(TypeCacheEntry *) * nkeys);
+ my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+ my_extra->n = nkeys;
+ }
+
+ /* Get TypeCacheEntry for each partition column. */
+ for (i = 0; i < nkeys; i++)
+ {
+ /* keys start from third argument of function. */
+ if (!PG_ARGISNULL(i + 2))
+ {
+ Oid valtype;
+
+ valtype = get_fn_expr_argtype(fcinfo->flinfo, (i + 2));
+ if (!OidIsValid(valtype))
+ elog(ERROR, "could not determine data type of satisfies_hash_partition() input");
+
+ /* Get the hash function. */
+ if (my_extra->typentry[i] == NULL ||
+ my_extra->typentry[i]->type_id != valtype)
+ {
+ my_extra->typentry[i] =
+ lookup_type_cache(valtype, TYPECACHE_HASH_PROC_FINFO);
+ if (!OidIsValid(my_extra->typentry[i]->hash_proc_finfo.fn_oid))
+ ereport(ERROR,
+ (errcode(ERRCODE_UNDEFINED_FUNCTION),
+ errmsg("could not identify a hash function for type %s",
+ format_type_be(valtype))));
+ }
+
+ values[i] = PG_GETARG_DATUM(i + 2);
+ isnull[i] = false;
+ partsupfunc[i] = my_extra->typentry[i]->hash_proc_finfo;
+ }
+ else
+ isnull[i] = true;
+ }
+
+ rowHash = cal_hash_value(partsupfunc, nkeys, values, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index cdcb949..a14c8f9 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13220,7 +13222,8 @@ 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;
@@ -13365,27 +13368,49 @@ 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.
- */
- if (!pelem->opclass)
+ if (strategy == PARTITION_STRATEGY_HASH)
{
- partopclass[attn] = GetDefaultOpClass(atttype, BTREE_AM_OID);
+ /* Identify a hash opclass to use */
+ if (!pelem->opclass)
+ {
+ partopclass[attn] = GetDefaultOpClass(atttype, HASH_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 (!OidIsValid(partopclass[attn]))
+ 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
+ partopclass[attn] = ResolveOpClass(pelem->opclass,
+ atttype,
+ "hash",
+ HASH_AM_OID);
}
else
- partopclass[attn] = ResolveOpClass(pelem->opclass,
- atttype,
- "btree",
- BTREE_AM_OID);
+ {
+ /*
+ * Identify a btree opclass to use. Currently, we use only btree
+ * operators, which seems enough for list and range partitioning.
+ */
+ if (!pelem->opclass)
+ {
+ partopclass[attn] = GetDefaultOpClass(atttype, BTREE_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.")));
+ }
+ else
+ partopclass[attn] = ResolveOpClass(pelem->opclass,
+ atttype,
+ "btree",
+ BTREE_AM_OID);
+ }
attn++;
}
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 35a237a..14bc6d7 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 21dfbb0..c6c2b6a 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 98f6768..808813b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 818d2c2..b5a6ad2 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -580,7 +580,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2645,8 +2646,36 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ n->remainder = defGetInt32(opt);
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("unrecognized hash partition bound specification \"%s\"",
+ opt->defname),
+ parser_errposition(opt->location)));
+ }
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2671,6 +2700,20 @@ ForValues:
}
;
+hash_partbound_elem:
+ NonReservedWord Iconst
+ {
+ $$ = makeDefElem($1, (Node *)makeInteger($2), @1);
+ }
+ ;
+
+hash_partbound:
+ hash_partbound_elem ',' hash_partbound_elem
+ {
+ $$ = list_make2($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 e187409..7aed36d 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,33 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("modulus must be greater than zero")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("remainder must be greater than or equal to zero")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("invalid bound specification for a hash partition"),
+ errhint("modulus 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 cbde1ff..af0d32b 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1653,6 +1653,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags,
switch (form->partstrat)
{
+ case PARTITION_STRATEGY_HASH:
+ appendStringInfo(&buf, "HASH");
+ break;
case PARTITION_STRATEGY_LIST:
if (!attrsOnly)
appendStringInfo(&buf, "LIST");
@@ -8619,6 +8622,14 @@ get_rule_expr(Node *node, deparse_context *context,
switch (spec->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ Assert(spec->modulus > 0 && spec->remainder >= 0);
+
+ 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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index e2a3351..195ea16 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2006,7 +2006,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 82562ad..660e8bd 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5468,6 +5468,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 2276" _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 e1d454a..4e8eb4f 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..5f0e6a8 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,57 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching list partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than zero
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than remainder
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+ERROR: invalid bound specification for a hash partition
+HINT: every modulus must be factor of next largest modulus
+DROP TABLE fail_part;
--
-- DETACH PARTITION
--
@@ -3287,12 +3339,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3434,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 dda0d7e..50a1b08 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
@@ -479,6 +474,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -505,6 +505,31 @@ 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 (unbounded);
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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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: invalid bound specification for a hash partition
+HINT: every modulus must be factor of next largest 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 (
@@ -512,6 +537,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 (
@@ -519,6 +546,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -526,6 +555,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -533,6 +564,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -591,6 +626,25 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: invalid bound specification for a hash partition
+HINT: modulus must be greater than zero
+-- 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: invalid bound specification for a hash partition
+HINT: modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -677,6 +731,8 @@ Number of partitions: 3 (Use \d+ to list them.)
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted, hpart_1, hpart_2;
+DROP TABLE hash_parted2, h2part_1, h2part_2, h2part_3, h2part_4;
-- 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 6f34b1c..fbfec10 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (HASHa, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..bcdb036 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching list partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 caf5ddb..3f1745f 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
@@ -451,6 +446,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 (
@@ -472,6 +469,21 @@ 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 (unbounded);
+-- 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
@@ -480,6 +492,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
@@ -487,6 +500,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -494,6 +508,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -502,6 +517,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -553,6 +569,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -606,6 +637,8 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- cleanup
DROP TABLE parted, list_parted, range_parted, list_parted2, range_parted2, range_parted3;
+DROP TABLE hash_parted, hpart_1, hpart_2;
+DROP TABLE hash_parted2, h2part_1, h2part_2, h2part_3, h2part_4;
-- 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 020854c..1ae8973 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,47 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Tue, May 2, 2017 at 7:01 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, May 2, 2017 at 9:01 PM, Jeff Davis <pgsql@j-davis.com> wrote:
1. Consider a partition-wise join of two hash-partitioned tables. If
that's a hash join, and we just use the hash opclass, we immediately
lose some useful bits of the hash function. Same for hash aggregation
where the grouping key is the partition key.Hmm, that could be a problem in some cases. I think there's probably
much less of a problem if the modulus isn't a power of two?
That's true, but it's awkward to describe that to users. And I think
most people would be inclined to use a power-of-two number of
partitions, perhaps coming from other systems.
To fix this, I think we need to include a salt in the hash API. Each
level of hashing can choose a random salt.Do you mean that we'd salt partitioning hashing differently from
grouping hashing which would be salted different from aggregation
hashing which, I suppose, would be salted differently from hash index
hashing?
Yes. The way I think about it is that choosing a new random salt is an
easy way to get a new hash function.
Or do you mean that you'd have to specify a salt when
creating a hash-partitioned table, and make sure it's the same across
all compatibly partitioned tables you might want to hash-join? That
latter sounds unappealing.
I don't see a reason to expose the salt to users. If we found a reason
in the future, we could, but it would create all of the problems you
are thinking about.
2. Consider a partition-wise join where the join keys are varchar(10)
and char(10). We can't do that join if we just use the existing hash
strategy, because 'foo' = 'foo ' should match, but those values
have different hashes when using the standard hash opclass.
...
You're basically describing what a hash opfamily already does, except
that we don't have a single opfamily that covers both varchar(10) and
char(10), nor do we have one that covers both int and numeric. We
have one that covers int2, int4, and int8, though. If somebody wanted
to make the ones you're suggesting, there's nothing preventing it,
although I'm not sure exactly how we'd encourage people to start using
the new one and deprecating the old one. We don't seem to have a good
infrastructure for that.
OK. I will propose new hash opfamilies for varchar/bpchar/text,
int2/4/8/numeric, and timestamptz/date.
One approach is to promote the narrower type to the wider type, and
then hash. The problem is that would substantially slow down the
hashing of integers, so then we'd need to use one hash opfamily for
partitioning and one for hashjoin, and it gets messy.
The other approach is to check if the wider type is within the domain
of the narrower type, and if so, *demote* the value and then hash. For
instance, '4.2'::numeric would hash the same as it does today, but
'4'::numeric would hash as an int2. I prefer this approach, and int8
already does something resembling it.
For timestamptz/date, it's not nearly as important.
My opinion is that we should work on this hashing infrastructure
first, and then support the DDL. If we get the hash functions right,
that frees us up to create better plans, with better push-downs, which
will be good for parallel query.I am opposed to linking the fate of this patch to multiple
independent, possibly large, possibly difficult, possibly
controversial enhancements to the hashing mechanism.
It's a little early in the v11 cycle to be having this argument.
Really what I'm saying is that a small effort now may save us a lot of
headache later.
Regards,
Jeff Davis
--
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, May 3, 2017 at 6:39 PM, amul sul <sulamul@gmail.com> wrote:
On Thu, Apr 27, 2017 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:
This is not yet a detailed review - I may be missing things, and
review and commentary from others is welcome. If there is no major
disagreement with the idea of moving forward using Amul's patch as a
base, then I will do a more detailed review of that patch (or,
hopefully, an updated version that addresses the above comments).
I agree that Amul's approach makes dump/restore feasible whereas
Nagata-san's approach makes that difficult. That is a major plus point
about Amul's patch. Also, it makes it possible to implement
Nagata-san's syntax, which is more user-friendly in future.
Here are some review comments after my initial reading of Amul's patch:
Hash partitioning will partition the data based on the hash value of the
partition key. Does that require collation? Should we throw an error/warning if
collation is specified in PARTITION BY clause?
+ int *indexes; /* Partition indexes; in case of hash
+ * partitioned table array length will be
+ * value of largest modulus, and for others
+ * one entry per member of the datums array
+ * (plus one if range partitioned table) */
This may be rewritten as "Partition indexes: For hash partitioned table the
number of indexes will be same as the largest modulus. For list partitioned
table the number of indexes will be same as the number of datums. For range
partitioned table the number of indexes will be number of datums plus one.".
You may be able to reword it to a shorter version, but essentially we will have
separate description for each strategy.
I guess, we need to change the comments for the other members too. For example
"datums" does not contain tuples with key->partnatts attributes for hash
partitions. It contains a tuple with two attributes, modulus and remainder. We
may not want to track null_index separately since rows with NULL partition key
will fit in the partition corresponding to the hash value of NULL. OR may be we
want to set null_index to partition which contains NULL values, if there is a
partition created for corresponding remainder, modulus pair and set has_null
accordingly. Accordingly we will need to update the comments.
cal_hash_value() may be renamed as calc_has_value() or compute_hash_value()?
Should we change the if .. else if .. construct in RelationBuildPartitionDesc()
to a switch case? There's very less chance that we will support a fourth
partitioning strategy, so if .. else if .. may be fine.
+ int mod = hbounds[i]->modulus,
+ place = hbounds[i]->remainder;
Although there are places in the code where we separate variable declaration
with same type by comma, most of the code declares each variable with the data
type on separate line. Should variable "place" be renamed as "remainder" since
that's what it is ultimately?
RelationBuildPartitionDesc() fills up mapping array but never uses it. In this
code the index into mapping array itself is the mapping so it doesn't need to
be maintained separately like list partiioning case. Similary next_index usage
looks unnecessary, although that probably improves readability, so may be fine.
+ * for p_p1: satisfies_hash_partition(2, 1, pkey, value)
+ * for p_p2: satisfies_hash_partition(4, 2, pkey, value)
+ * for p_p3: satisfies_hash_partition(8, 0, pkey, value)
+ * for p_p4: satisfies_hash_partition(8, 4, pkey, value)
What the function builds is satisfies_hash_partition(2, 1, pkey). I don't see
code to add value as an argument to the function. Is that correct?
+ int modulus = DatumGetInt32(datum);
May be you want to rename this variable to greatest_modulus like in the other
places.
+ Assert(spec->modulus > 0 && spec->remainder >= 0);
I liked this assertion. Do you want to add spec->modulus > spec->reminder also
here?
+ char *strategy; /* partitioning strategy
+ ('hash', 'list' or 'range') */
We need the second line to start with '*'
+-- check validation when attaching list partitions
Do you want to say "hash" instead of "list" here?
I think we need to explain the reasoning behind this syntax somewhere
as a README or in the documentation or in the comments. Otherwise it's
difficult to understand how various pieces of code are related.
This is not full review. I am still trying to understand how the hash
partitioning implementation fits with list and range partitioning. I
am going to continue to review this patch further.
--
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, May 4, 2017 at 1:44 AM, Jeff Davis <pgsql@j-davis.com> wrote:
Hmm, that could be a problem in some cases. I think there's probably
much less of a problem if the modulus isn't a power of two?That's true, but it's awkward to describe that to users. And I think
most people would be inclined to use a power-of-two number of
partitions, perhaps coming from other systems.
Yeah, true.
To fix this, I think we need to include a salt in the hash API. Each
level of hashing can choose a random salt.Do you mean that we'd salt partitioning hashing differently from
grouping hashing which would be salted different from aggregation
hashing which, I suppose, would be salted differently from hash index
hashing?Yes. The way I think about it is that choosing a new random salt is an
easy way to get a new hash function.
OK. One problem, though, is we don't quite have the opclass
infrastructure for this. A hash opclass's support function is
expected to take one argument, a value of the data type at issue. The
first idea that occurred to me was to allow an optional second
argument which would be a seed, but that seems like it would require
extensive changes to all of the datatype-specific hash functions and
some of them would probably emerge noticeably slower. If a function
is just calling hash_uint32 right now then I don't see how we're going
to replace that with something more complex that folds in a salt
without causing performance to drop. Even just the cost of unpacking
the extra argument might be noticeable.
Another alternative would be to be to add one additional, optional
hash opclass support function which takes a value of the type in
question as one argument and a seed as a second argument. That seems
like it might work OK. Existing code can use the existing support
function 1 with no change, and hash partitioning can use support
function 2.
Or do you mean that you'd have to specify a salt when
creating a hash-partitioned table, and make sure it's the same across
all compatibly partitioned tables you might want to hash-join? That
latter sounds unappealing.I don't see a reason to expose the salt to users. If we found a reason
in the future, we could, but it would create all of the problems you
are thinking about.
Right, OK.
You're basically describing what a hash opfamily already does, except
that we don't have a single opfamily that covers both varchar(10) and
char(10), nor do we have one that covers both int and numeric. We
have one that covers int2, int4, and int8, though. If somebody wanted
to make the ones you're suggesting, there's nothing preventing it,
although I'm not sure exactly how we'd encourage people to start using
the new one and deprecating the old one. We don't seem to have a good
infrastructure for that.OK. I will propose new hash opfamilies for varchar/bpchar/text,
int2/4/8/numeric, and timestamptz/date.
Cool! I have no idea how we'll convert from the old ones to the new
ones without breaking things but I agree that it would be nicer if it
were like that rather than the way it is now.
One approach is to promote the narrower type to the wider type, and
then hash. The problem is that would substantially slow down the
hashing of integers, so then we'd need to use one hash opfamily for
partitioning and one for hashjoin, and it gets messy.
Yes, that sounds messy.
The other approach is to check if the wider type is within the domain
of the narrower type, and if so, *demote* the value and then hash. For
instance, '4.2'::numeric would hash the same as it does today, but
'4'::numeric would hash as an int2. I prefer this approach, and int8
already does something resembling it.
Sounds reasonable.
It's a little early in the v11 cycle to be having this argument.
Really what I'm saying is that a small effort now may save us a lot of
headache later.
Well, that's fair enough. My concern is basically that it may the
other way around: a large effort to save a small headache later. I
agree that it's probably a good idea to figure out a way to salt the
hash function so that we don't end up with this and partitionwise join
interacting badly, but I don't see the other issues as being very
critical. I don't have any evidence that there's a big need to
replace our hash functions with new ones, and over on the
partitionwise join thread we gave up on the idea of a cross-type
partitionwise join. It wouldn't be particularly common (or sensible,
really) even if we ended up supporting it.
--
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, May 10, 2017 at 8:34 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Hash partitioning will partition the data based on the hash value of the
partition key. Does that require collation? Should we throw an error/warning if
collation is specified in PARTITION BY clause?
Collation is only relevant for ordering, not equality. Since hash
opclasses provide only equality, not ordering, it's not relevant here.
I'm not sure whether we should error out if it's specified or just
silently ignore it. Maybe an ERROR is a good idea? But not sure.
--
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, May 3, 2017 at 9:09 AM, amul sul <sulamul@gmail.com> wrote:
Fixed in the attached version.
+[ PARTITION BY { HASH | RANGE | LIST } ( { <replaceable
class="parameter">column_name</replaceable> | ( <replaceable
class="parameter">expression</replaceable> ) } [ COLLATE <replaceable
In the department of severe nitpicking, I would have expected this to
either use alphabetical order (HASH | LIST | RANGE) or to add the new
method at the end on the theory that we probably did the important
ones first (RANGE | LIST | HASH).
+ WITH ( MODULUS <replaceable class="PARAMETER">value</replaceable>,
REMAINDER <replaceable class="PARAMETER">value</replaceable> ) }
Maybe value -> modulus and value -> remainder?
<para>
+ When creating a hash partition, <literal>MODULUS</literal> should be
+ greater than zero and <literal>REMAINDER</literal> should be greater than
+ or equal to zero. Every <literal>MODULUS</literal> must be a factor of
+ the next larger modulus.
[ ... and it goes on from there ... ]
This paragraph is fairly terrible, because it's a design spec that I
wrote, not an explanation intended for users. Here's an attempt to
improve it:
===
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.
===
+CREATE TABLE postal_code (
+ code int not null,
+ city_id bigint not null,
+ address text
+) PARTITION BY HASH (code);
It would be fairly silly to hash-partition the postal_code table,
because there aren't enough postal codes to justify it. Maybe make
this a lineitem or order table, and partition on the order number.
Also, extend the example to show creating 4 partitions with modulus 4.
+ if (spec->strategy != PARTITION_STRATEGY_HASH)
+ elog(ERROR, "invalid strategy in partition bound spec");
I think this should be an ereport() if it can happen or an Assert() if
it's supposed to be prevented by the grammar.
+ if (!(datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+ true, sizeof(int)) &&
It doesn't seem necessary to use datumIsEqual() here. You know the
datums are pass-by-value, so why not just use == ? I'd include a
comment but I don't think using datumIsEqual() adds anything here
except unnecessary complexity. More broadly, I wonder why we're
cramming this into the datums arrays instead of just adding another
field to PartitionBoundInfoData that is only used by hash
partitioning.
/*
+ * 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
+ * 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. However, you could
simultaneously
+ * use modulus 4, modulus 8, modulus 16, and modulus 32 if
+ * you wished, because each modulus is a factor of the next
+ * larger one. You could also use modulus 10, modulus 20,
+ * and modulus 60. But you could not use modulus 10,
+ * modulus 15, and modulus 60 for the same reason.
+ */
I think just the first sentence is fine here; I'd nuke the rest of this.
The block that follows could be merged into the surrounding block.
There's no need to increase the indentation level here, so let's not.
I also suspect that the code itself is wrong. There are two ways a
modulus can be invalid: it can either fail to be a multiple of the
next lower-modulus, or it can fail to be a factor of the next-higher
modulus. I think your code only checks the latter. So for example,
if the current modulus list is (4, 36), your code would correctly
disallow 3 because it's not a factor of 4 and would correctly disallow
23 because it's not a factor of 36, but it looks to me like it would
allow 9 because that's a factor of 36. However, then the list would be
(4, 9, 36), and 4 is not a factor of 9.
+ greatest_modulus = DatumGetInt32(datums[ndatums - 1][0]);
Here, insert: /* 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. */
+ place = spec->remainder;
+ if (place >= greatest_modulus)
+ place = place % greatest_modulus;
Here, insert: /* Check every potentially-conflicting remainder. */
+ do
+ {
+ if (boundinfo->indexes[place] != -1)
+ {
+ overlap = true;
+ with = boundinfo->indexes[place];
+ break;
+ }
+ place = place + spec->modulus;
Maybe use += ?
+ } while (place < greatest_modulus);
+ * Used when sorting hash bounds across all hash modulus
+ * for hash partitioning
This is not a very descriptive comment. Maybe /* We sort hash bounds
by modulus, then by remainder. */
+cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values, bool *isnull)
I agree with Ashutosh's critique of this name.
+ /*
+ * Cache hash function information, similar to how record_eq() caches
+ * equality operator information. (Perhaps no SQL syntax could cause
+ * PG_NARGS()/nkeys to change between calls through the same FmgrInfo.
+ * Checking nkeys here is just defensiveness.)
+ */
Unless I'm missing something, this comment does not actually describe
what the code does. Each call to the function repeats the same
TypeCacheEntry lookups. I'm not actually sure whether caching here
can actually help - is there any situation in which the same FmgrInfo
will get used repeatedly here? But if it is possible then this code
fails to achieve its intended objective.
Another problem with this code is that, unless I'm missing something,
it completely ignores the opclass the user specified and just looks up
the default hash opclass. I think you should create a non-default
hash opclass for some data type -- maybe create one for int4 that just
returns the input value unchanged -- and test that the specifying
default hash opclass routes tuples according to hash_uint32(val) %
modulus while specifying your customer opclass routes tuples according
to val % modulus.
Unless I'm severely misunderstanding the situation this code is
seriously undertested.
+ * Identify a btree opclass to use. Currently, we use only btree
+ * operators, which seems enough for list and range partitioning.
This comment is false, right?
+ appendStringInfoString(buf, "FOR VALUES");
+ appendStringInfo(buf, " WITH (modulus %d,
remainder %d)",
+ spec->modulus, spec->remainder);
You could combine these.
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH
(modulus 0, remainder 1);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than zero
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH
(modulus 8, remainder 8);
+ERROR: invalid bound specification for a hash partition
+HINT: modulus must be greater than remainder
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH
(modulus 3, remainder 2);
+ERROR: invalid bound specification for a hash partition
+HINT: every modulus must be factor of next largest modulus
It seems like you could merge the hint back into the error:
ERROR: hash partition modulus must be greater than 0
ERROR: hash partition remainder must be less than modulus
ERROR: every hash partition modulus must be a factor of the next larger modulus
+DETAIL: Partition key of the failing row contains (HASHa, b) = (c, 5).
That's obviously garbled somehow.
+hash_partbound_elem:
+ NonReservedWord Iconst
+ {
+ $$ = makeDefElem($1, (Node *)makeInteger($2), @1);
+ }
+ ;
+
+hash_partbound:
+ hash_partbound_elem ',' hash_partbound_elem
+ {
+ $$ = list_make2($1, $3);
+ }
+ ;
I don't think that it's the grammar's job to enforce that exactly two
options are present. It should allow any number of options, and some
later code, probably during parse analysis, should check that the ones
you need are present and that there are no invalid ones. See the code
for EXPLAIN, VACUUM, etc.
Regarding the test cases, I think that you've got a lot of tests for
failure scenarios (which is good) but not enough for success
scenarios. For example, you test that inserting a row into the wrong
hash partition fails, but not (unless I missed it) that tuple routing
succeeds. I think it would be good to have a test where you insert
1000 or so rows into a hash partitioned table just to see it all work.
Also, you haven't done anything about the fact that constraint
exclusion doesn't work for hash partitioned tables, a point I raised
in /messages/by-id/CA+Tgmob7RsN5A=ehgYbLPx--c5CmptrK-dB=Y-v--o+TKyfteA@mail.gmail.com
and which I still think is quite important. I think that to have a
committable patch for this feature that would have to be addressed.
--
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, May 3, 2017 at 6:39 PM, amul sul <sulamul@gmail.com> wrote:
On Thu, Apr 27, 2017 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:
I spent some time today looking at these patches. It seems like there
is some more work still needed here to produce something committable
regardless of which way we go, but I am inclined to think that Amul's
patch is a better basis for work going forward than Nagata-san's
patch. Here are some general comments on the two patches:Thanks for your time.
[...]
- Neither patch contains any documentation updates, which is bad.
Fixed in the attached version.
I have done an intial review of the patch and I have some comments. I
will continue the review
and testing and report the results soon
-----
Patch need to be rebased
----
if (key->strategy == PARTITION_STRATEGY_RANGE)
{
/* Disallow nulls in the range partition key of the tuple */
for (i = 0; i < key->partnatts; i++)
if (isnull[i])
ereport(ERROR,
(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("range partition key of row contains null")));
}
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?
----
RangeDatumContent **content;/* what's contained in each range bound datum?
* (see the above enum); NULL for list
* partitioned tables */
This will be NULL for hash as well we need to change the comments.
-----
bool has_null; /* Is there a null-accepting partition? false
* for range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
Comments needs to be changed for these two members as well
----
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+ int modulus;
+ int remainder;
+ int index;
+} PartitionHashBound;
It will good to add some comments to explain the structure members
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 Thu, May 11, 2017 at 12:02 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?
I think it should.
Actually, I think that not supporting nulls for range partitioning may
have been a fairly bad decision.
--
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/05/12 10:42, Robert Haas wrote:
On Thu, May 11, 2017 at 12:02 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?I think it should.
Actually, I think that not supporting nulls for range partitioning may
have been a fairly bad decision.
I think the relevant discussion concluded [1]/messages/by-id/CA+TgmoZN_Zf7MBb48O66FAJgFe0S9_NkLVeQNBz6hsxb6Og93w@mail.gmail.com that way, because we
couldn't decide which interface to provide for specifying where NULLs are
placed or because we decided to think about it later.
Thanks,
Amit
[1]: /messages/by-id/CA+TgmoZN_Zf7MBb48O66FAJgFe0S9_NkLVeQNBz6hsxb6Og93w@mail.gmail.com
/messages/by-id/CA+TgmoZN_Zf7MBb48O66FAJgFe0S9_NkLVeQNBz6hsxb6Og93w@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 Thu, May 11, 2017 at 10:15 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
On 2017/05/12 10:42, Robert Haas wrote:
On Thu, May 11, 2017 at 12:02 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?I think it should.
Actually, I think that not supporting nulls for range partitioning may
have been a fairly bad decision.I think the relevant discussion concluded [1] that way, because we
couldn't decide which interface to provide for specifying where NULLs are
placed or because we decided to think about it later.
Yeah, but I have a feeling that marking the columns NOT NULL is going
to make it really hard to support that in the future when we get the
syntax hammered out. If it had only affected the partition
constraints that'd be different.
--
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/05/12 11:20, Robert Haas wrote:
On Thu, May 11, 2017 at 10:15 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:On 2017/05/12 10:42, Robert Haas wrote:
On Thu, May 11, 2017 at 12:02 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?I think it should.
Actually, I think that not supporting nulls for range partitioning may
have been a fairly bad decision.I think the relevant discussion concluded [1] that way, because we
couldn't decide which interface to provide for specifying where NULLs are
placed or because we decided to think about it later.Yeah, but I have a feeling that marking the columns NOT NULL is going
to make it really hard to support that in the future when we get the
syntax hammered out. If it had only affected the partition
constraints that'd be different.
So, adding keycol IS NOT NULL (like we currently do for expressions) in
the implicit partition constraint would be more future-proof than
generating an actual catalogued NOT NULL constraint on the keycol? I now
tend to think it would be better. Directly inserting into a range
partition with a NULL value for a column currently generates a "null value
in column \"%s\" violates not-null constraint" instead of perhaps more
relevant "new row for relation \"%s\" violates partition constraint".
That said, we *do* document the fact that a NOT NULL constraint is added
on range key columns, but we might as well document instead that we don't
currently support routing tuples with NULL values in the partition key
through a range-partitioned table and so NULL values cause error.
Can we still decide to do that instead?
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 Fri, May 12, 2017 at 7:12 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, May 11, 2017 at 12:02 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?I think it should.
+1
As long as we can hash a NULL value, we should place a value with NULL
key in the corresponding partition, most probably the one with
remainder 0.
--
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, May 12, 2017 at 8:08 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
On 2017/05/12 11:20, Robert Haas wrote:
On Thu, May 11, 2017 at 10:15 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:On 2017/05/12 10:42, Robert Haas wrote:
On Thu, May 11, 2017 at 12:02 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?I think it should.
Actually, I think that not supporting nulls for range partitioning may
have been a fairly bad decision.I think the relevant discussion concluded [1] that way, because we
couldn't decide which interface to provide for specifying where NULLs are
placed or because we decided to think about it later.Yeah, but I have a feeling that marking the columns NOT NULL is going
to make it really hard to support that in the future when we get the
syntax hammered out. If it had only affected the partition
constraints that'd be different.So, adding keycol IS NOT NULL (like we currently do for expressions) in
the implicit partition constraint would be more future-proof than
generating an actual catalogued NOT NULL constraint on the keycol? I now
tend to think it would be better. Directly inserting into a range
partition with a NULL value for a column currently generates a "null value
in column \"%s\" violates not-null constraint" instead of perhaps more
relevant "new row for relation \"%s\" violates partition constraint".
That said, we *do* document the fact that a NOT NULL constraint is added
on range key columns, but we might as well document instead that we don't
currently support routing tuples with NULL values in the partition key
through a range-partitioned table and so NULL values cause error.
in get_partition_for_tuple() we have
if (key->strategy == PARTITION_STRATEGY_RANGE)
{
/* Disallow nulls in the range partition key of the tuple */
for (i = 0; i < key->partnatts; i++)
if (isnull[i])
ereport(ERROR,
(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("range partition key of row contains null")));
}
Instead of throwing an error here, we should probably return -1 and
let the error be ""no partition of relation \"%s\" found for row",
which is the real error, not having a partition which can accept NULL.
If in future we decide to support NULL values in partition keys, we
need to just remove above code from get_partition_for_tuple() and
everything will work as is. I am assuming that we don't add any
implicit/explicit NOT NULL constraint right now.
--
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 2017/05/12 14:24, Ashutosh Bapat wrote:
On Fri, May 12, 2017 at 8:08 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:On 2017/05/12 11:20, Robert Haas wrote:
Yeah, but I have a feeling that marking the columns NOT NULL is going
to make it really hard to support that in the future when we get the
syntax hammered out. If it had only affected the partition
constraints that'd be different.So, adding keycol IS NOT NULL (like we currently do for expressions) in
the implicit partition constraint would be more future-proof than
generating an actual catalogued NOT NULL constraint on the keycol? I now
tend to think it would be better. Directly inserting into a range
partition with a NULL value for a column currently generates a "null value
in column \"%s\" violates not-null constraint" instead of perhaps more
relevant "new row for relation \"%s\" violates partition constraint".
That said, we *do* document the fact that a NOT NULL constraint is added
on range key columns, but we might as well document instead that we don't
currently support routing tuples with NULL values in the partition key
through a range-partitioned table and so NULL values cause error.in get_partition_for_tuple() we have
if (key->strategy == PARTITION_STRATEGY_RANGE)
{
/* Disallow nulls in the range partition key of the tuple */
for (i = 0; i < key->partnatts; i++)
if (isnull[i])
ereport(ERROR,
(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("range partition key of row contains null")));
}Instead of throwing an error here, we should probably return -1 and
let the error be ""no partition of relation \"%s\" found for row",
which is the real error, not having a partition which can accept NULL.
If in future we decide to support NULL values in partition keys, we
need to just remove above code from get_partition_for_tuple() and
everything will work as is. I am assuming that we don't add any
implicit/explicit NOT NULL constraint right now.
We *do* actually, for real columns:
create table p (a int) partition by range (a);
\d p
Table "public.p"
Column | Type | Collation | Nullable | Default
--------+---------+-----------+----------+---------
a | integer | | not null |
Partition key: RANGE (a)
For expression keys, we emit IS NOT NULL as part of the implicit partition
constraint. The above check for NULL is really for the expressions,
because if any simple columns of the key contain NULL, they will fail the
NOT NULL constraint itself (with that error message). As I said in my
previous message, I'm thinking that emitting IS NOT NULL as part of the
implicit partition constraint might be better instead of adding it as a
NOT NULL constraint, that is, for the simple column keys; we already do
that for the expression keys for which we cannot add the NOT NULL
constraint anyway.
The way things are currently, error messages generated when a row with
NULL in the range partition key is *directly* into the partition looks a
bit inconsistent, depending on whether the target key is a simple column
or expression:
create table p (a int, b int) partition by range (a, abs(b));
create table p1 partition of p for values from (1, 1) to (1, 10);
insert into p1 values (NULL, NULL);
ERROR: null value in column "a" violates not-null constraint
DETAIL: Failing row contains (null, null).
insert into p1 values (1, NULL);
ERROR: new row for relation "p1" violates partition constraint
DETAIL: Failing row contains (1, null).
It would be nice if both said "violates partition constraint".
BTW, note that this is independent of your suggestion to emit "partition
not found" message instead of the "no NULLs allowed in the range partition
key" message, which seems fine to me to implement.
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 Wed, May 10, 2017 at 6:04 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Wed, May 3, 2017 at 6:39 PM, amul sul <sulamul@gmail.com> wrote:
On Thu, Apr 27, 2017 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:
This is not yet a detailed review - I may be missing things, and
review and commentary from others is welcome. If there is no major
disagreement with the idea of moving forward using Amul's patch as a
base, then I will do a more detailed review of that patch (or,
hopefully, an updated version that addresses the above comments).I agree that Amul's approach makes dump/restore feasible whereas
Nagata-san's approach makes that difficult. That is a major plus point
about Amul's patch. Also, it makes it possible to implement
Nagata-san's syntax, which is more user-friendly in future.Here are some review comments after my initial reading of Amul's patch:
Hash partitioning will partition the data based on the hash value of the
partition key. Does that require collation? Should we throw an error/warning if
collation is specified in PARTITION BY clause?+ int *indexes; /* Partition indexes; in case of hash + * partitioned table array length will be + * value of largest modulus, and for others + * one entry per member of the datums array + * (plus one if range partitioned table) */ This may be rewritten as "Partition indexes: For hash partitioned table the number of indexes will be same as the largest modulus. For list partitioned table the number of indexes will be same as the number of datums. For range partitioned table the number of indexes will be number of datums plus one.". You may be able to reword it to a shorter version, but essentially we will have separate description for each strategy.
Okay, will fix this.
I guess, we need to change the comments for the other members too. For example
"datums" does not contain tuples with key->partnatts attributes for hash
partitions. It contains a tuple with two attributes, modulus and remainder. We
may not want to track null_index separately since rows with NULL partition key
will fit in the partition corresponding to the hash value of NULL. OR may be we
want to set null_index to partition which contains NULL values, if there is a
partition created for corresponding remainder, modulus pair and set has_null
accordingly. Accordingly we will need to update the comments.cal_hash_value() may be renamed as calc_has_value() or compute_hash_value()?
Okay, will rename to compute_hash_value().
Should we change the if .. else if .. construct in RelationBuildPartitionDesc()
to a switch case? There's very less chance that we will support a fourth
partitioning strategy, so if .. else if .. may be fine.+ int mod = hbounds[i]->modulus, + place = hbounds[i]->remainder; Although there are places in the code where we separate variable declaration with same type by comma, most of the code declares each variable with the data type on separate line. Should variable "place" be renamed as "remainder" since that's what it is ultimately?
Okay, will rename "place" to "remainder".
RelationBuildPartitionDesc() fills up mapping array but never uses it. In this
Agreed, mapping array is not that much useful but not useless, it
required at the end of RelationBuildPartitionDesc() while assigning
OIDs to result->oids, see for-loop just before releasing mapping
memory.
code the index into mapping array itself is the mapping so it doesn't need to
be maintained separately like list partiioning case. Similary next_index usage
looks unnecessary, although that probably improves readability, so may be fine.
Anyway, will remove uses of "next_index".
+ * for p_p1: satisfies_hash_partition(2, 1, pkey, value) + * for p_p2: satisfies_hash_partition(4, 2, pkey, value) + * for p_p3: satisfies_hash_partition(8, 0, pkey, value) + * for p_p4: satisfies_hash_partition(8, 4, pkey, value) What the function builds is satisfies_hash_partition(2, 1, pkey). I don't see code to add value as an argument to the function. Is that correct?
Sorry for confusion, "pkey" & "value" are the column of table in the
give example.
Renamed those column name to "a" & "b".
+ int modulus = DatumGetInt32(datum);
May be you want to rename this variable to greatest_modulus like in the other
places.
Okay, will fix this.
+ Assert(spec->modulus > 0 && spec->remainder >= 0);
I liked this assertion. Do you want to add spec->modulus > spec->reminder also
here?
Okay, will add this too.
+ char *strategy; /* partitioning strategy + ('hash', 'list' or 'range') */We need the second line to start with '*'
+-- check validation when attaching list partitions
Do you want to say "hash" instead of "list" here?
You are correct, will fix this too.
I think we need to explain the reasoning behind this syntax somewhere
as a README or in the documentation or in the comments. Otherwise it's
difficult to understand how various pieces of code are related.
Not sure about README, I think we should focus on documentation & code
comments first, and then think about developer perspective README if
hash partitioning logic is too difficult to understand .
This is not full review. I am still trying to understand how the hash
partitioning implementation fits with list and range partitioning. I
am going to continue to review this patch further.
Thanks a lots for your help.
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
Hi,
Please find the following updated patches attached:
0001-Cleanup.patch : Does some cleanup and code refactoring required
for hash partition patch. Otherwise, there will be unnecessary diff in
0002 patch
0002-hash-partitioning_another_design-v3.patch: Addressed review
comments given by Ashutosh and Robert.
On Wed, May 10, 2017 at 11:39 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, May 3, 2017 at 9:09 AM, amul sul <sulamul@gmail.com> wrote:
Fixed in the attached version.
+[ PARTITION BY { HASH | RANGE | LIST } ( { <replaceable
class="parameter">column_name</replaceable> | ( <replaceable
class="parameter">expression</replaceable> ) } [ COLLATE <replaceableIn the department of severe nitpicking, I would have expected this to
either use alphabetical order (HASH | LIST | RANGE) or to add the new
method at the end on the theory that we probably did the important
ones first (RANGE | LIST | HASH).
Fixed in the attached version.
+ WITH ( MODULUS <replaceable class="PARAMETER">value</replaceable>,
REMAINDER <replaceable class="PARAMETER">value</replaceable> ) }Maybe value -> modulus and value -> remainder?
Fixed in the attached version.
<para> + When creating a hash partition, <literal>MODULUS</literal> should be + greater than zero and <literal>REMAINDER</literal> should be greater than + or equal to zero. Every <literal>MODULUS</literal> must be a factor of + the next larger modulus. [ ... and it goes on from there ... ]This paragraph is fairly terrible, because it's a design spec that I
wrote, not an explanation intended for users. Here's an attempt to
improve it:===
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.
===
Thanks a lot, added in attached version.
+CREATE TABLE postal_code ( + code int not null, + city_id bigint not null, + address text +) PARTITION BY HASH (code);It would be fairly silly to hash-partition the postal_code table,
because there aren't enough postal codes to justify it. Maybe make
this a lineitem or order table, and partition on the order number.
Also, extend the example to show creating 4 partitions with modulus 4.
Understood, added order table example.
+ if (spec->strategy != PARTITION_STRATEGY_HASH) + elog(ERROR, "invalid strategy in partition bound spec");I think this should be an ereport() if it can happen or an Assert() if
it's supposed to be prevented by the grammar.
Used Assert() in the attach version patch, also changed same for RANGE
and LIST in 0001- cleanup patch.
+ if (!(datumIsEqual(b1->datums[i][0], b2->datums[i][0], + true, sizeof(int)) &&It doesn't seem necessary to use datumIsEqual() here. You know the
datums are pass-by-value, so why not just use == ? I'd include a
comment but I don't think using datumIsEqual() adds anything here
except unnecessary complexity. More broadly, I wonder why we're
cramming this into the datums arrays instead of just adding another
field to PartitionBoundInfoData that is only used by hash
partitioning.
Fixed in the attached version.
/* + * 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 + * 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. However, you could simultaneously + * use modulus 4, modulus 8, modulus 16, and modulus 32 if + * you wished, because each modulus is a factor of the next + * larger one. You could also use modulus 10, modulus 20, + * and modulus 60. But you could not use modulus 10, + * modulus 15, and modulus 60 for the same reason. + */I think just the first sentence is fine here; I'd nuke the rest of this.
Fixed in the attached version.
The block that follows could be merged into the surrounding block.
There's no need to increase the indentation level here, so let's not.
I also suspect that the code itself is wrong. There are two ways a
modulus can be invalid: it can either fail to be a multiple of the
next lower-modulus, or it can fail to be a factor of the next-higher
modulus. I think your code only checks the latter. So for example,
if the current modulus list is (4, 36), your code would correctly
disallow 3 because it's not a factor of 4 and would correctly disallow
23 because it's not a factor of 36, but it looks to me like it would
allow 9 because that's a factor of 36. However, then the list would be
(4, 9, 36), and 4 is not a factor of 9.
This case is already handled in previous patch and similar regression
test does exists in create_table.sql, see this in v2 patch.
+-- 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);
+ greatest_modulus = DatumGetInt32(datums[ndatums - 1][0]);
Here, insert: /* 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. */+ place = spec->remainder; + if (place >= greatest_modulus) + place = place % greatest_modulus;
Fixed in the attached version.
Here, insert: /* Check every potentially-conflicting remainder. */
+ do + { + if (boundinfo->indexes[place] != -1) + { + overlap = true; + with = boundinfo->indexes[place]; + break; + } + place = place + spec->modulus;Maybe use += ?
Fixed.
+ } while (place < greatest_modulus);
+ * Used when sorting hash bounds across all hash modulus + * for hash partitioningThis is not a very descriptive comment. Maybe /* We sort hash bounds
by modulus, then by remainder. */
Fixed.
+cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values, bool *isnull)
I agree with Ashutosh's critique of this name.
Fixed.
+ /* + * Cache hash function information, similar to how record_eq() caches + * equality operator information. (Perhaps no SQL syntax could cause + * PG_NARGS()/nkeys to change between calls through the same FmgrInfo. + * Checking nkeys here is just defensiveness.) + */Unless I'm missing something, this comment does not actually describe
what the code does. Each call to the function repeats the same
TypeCacheEntry lookups. I'm not actually sure whether caching here
can actually help - is there any situation in which the same FmgrInfo
will get used repeatedly here? But if it is possible then this code
fails to achieve its intended objective.
This code is no longer exists in new satisfies_hash_partition() code.
Another problem with this code is that, unless I'm missing something,
it completely ignores the opclass the user specified and just looks up
the default hash opclass. I think you should create a non-default
hash opclass for some data type -- maybe create one for int4 that just
returns the input value unchanged -- and test that the specifying
default hash opclass routes tuples according to hash_uint32(val) %
modulus while specifying your customer opclass routes tuples according
to val % modulus.Unless I'm severely misunderstanding the situation this code is
seriously undertested.
You are correct, I've missed to opclass handling. Fixed in the
attached version, and added same case regression test.
+ * Identify a btree opclass to use. Currently, we use only btree + * operators, which seems enough for list and range partitioning.This comment is false, right?
Not really, this has been re-added due to indentation change.
+ appendStringInfoString(buf, "FOR VALUES"); + appendStringInfo(buf, " WITH (modulus %d, remainder %d)", + spec->modulus, spec->remainder);You could combine these.
I am not sure about this, I've used same code style exist in
get_rule_expr() for range and list. Do you want me to change this for
other partitioning as well?
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1); +ERROR: invalid bound specification for a hash partition +HINT: modulus must be greater than zero +ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8); +ERROR: invalid bound specification for a hash partition +HINT: modulus must be greater than remainder +ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2); +ERROR: invalid bound specification for a hash partition +HINT: every modulus must be factor of next largest modulusIt seems like you could merge the hint back into the error:
ERROR: hash partition modulus must be greater than 0
ERROR: hash partition remainder must be less than modulus
ERROR: every hash partition modulus must be a factor of the next larger modulus
Added same in the attached version. Thanks again.
+DETAIL: Partition key of the failing row contains (HASHa, b) = (c, 5).
That's obviously garbled somehow.
Oops. Fixed in the attached version.
+hash_partbound_elem: + NonReservedWord Iconst + { + $$ = makeDefElem($1, (Node *)makeInteger($2), @1); + } + ; + +hash_partbound: + hash_partbound_elem ',' hash_partbound_elem + { + $$ = list_make2($1, $3); + } + ;I don't think that it's the grammar's job to enforce that exactly two
options are present. It should allow any number of options, and some
later code, probably during parse analysis, should check that the ones
you need are present and that there are no invalid ones. See the code
for EXPLAIN, VACUUM, etc.
Tried to fixed in the attached version.
Regarding the test cases, I think that you've got a lot of tests for
failure scenarios (which is good) but not enough for success
scenarios. For example, you test that inserting a row into the wrong
hash partition fails, but not (unless I missed it) that tuple routing
succeeds. I think it would be good to have a test where you insert
1000 or so rows into a hash partitioned table just to see it all work.
I am quite unsure about this test, now sure how can we verify correct
tuple routing?
Also, you haven't done anything about the fact that constraint
exclusion doesn't work for hash partitioned tables, a point I raised
in /messages/by-id/CA+Tgmob7RsN5A=ehgYbLPx--c5CmptrK-dB=Y-v--o+TKyfteA@mail.gmail.com
and which I still think is quite important. I think that to have a
committable patch for this feature that would have to be addressed.
Do you mean, we should come up with special handling(pre-pruning) for
hash partitioning or modify constraints exclusion so that it will
handle hash partition expression and cases that you have discussed in
thread[1] as well? I was under the impression that we might going to
have this as a separate feature proposal.
1]. /messages/by-id/CA+TgmoaE9NZ_RiqZQLp2aJXPO4E78QxkQYL-FR2zCDop96Ahdg@mail.gmail.com
Regards,
Amul Sul
Attachments:
0001-Cleanup.patchapplication/octet-stream; name=0001-Cleanup.patchDownload
From 91a6f38b06e88345b1b07b712f316abba767faff Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 12 May 2017 13:53:55 +0530
Subject: [PATCH 1/2] Cleanup
Code refactoring required for hash partitioning patch v3
---
src/backend/catalog/partition.c | 82 ++++++++++++++++++++--------------------
src/backend/commands/tablecmds.c | 7 ++--
2 files changed, 46 insertions(+), 43 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 8641ae1..f42f287 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -244,8 +244,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -326,8 +325,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1674,10 +1672,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1719,40 +1715,46 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting list
+ * partition exists.
+ */
+ if (isnull[0])
+ cur_index = partdesc->boundinfo->null_index;
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -1760,12 +1762,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);
}
/*
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index cdcb949..49bb4d7 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13220,7 +13220,8 @@ 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;
--
2.6.2
0002-hash-partitioning_another_design-v3.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v3.patchDownload
From 91522ae67843082262d829719c97ca66eb9d546c Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 12 May 2017 14:25:12 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v3
v3:
Fixes w.r.t. Ashutosh's and Robert's review comments.
v2:
Regression test updated.
Documentation added.
Added tab complition for FOR VALUES WITH
v1:
Initial patch
---
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 57 +++-
src/backend/catalog/partition.c | 530 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 87 +++--
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 | 61 +++-
src/backend/parser/parse_utilcmd.c | 147 +++++---
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 ++++
src/test/regress/expected/create_table.out | 63 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 ++++
src/test/regress/sql/create_table.sql | 44 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 ++
22 files changed, 1191 insertions(+), 157 deletions(-)
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..1c57e52 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -1591,6 +1615,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
@@ -1641,6 +1675,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 f42f287..a86f41c 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,20 @@
* 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.
+ * In the case of hash partitioning, datums is a 2-D array, stores modulus and
+ * remainder values at datums[x][0] and datums[x][1] respectively for each
+ * partition in the ascending order.
*
* 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +84,20 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums; /* For hash partitioned table, array of
+ * modulus and remainder. For range and list
+ * partitioned table, array of datum-tuples
+ * with key->partnatts datums each */
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,11 +129,13 @@ 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,
void *arg);
+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 Oid get_partition_operator(PartitionKey key, int col,
@@ -140,6 +158,12 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 compute_hash_value(FmgrInfo *partsupfunc, int nkeys,
+ Datum *values, bool *isnull);
+
+/* SQL-callable function for use in hash partition CHECK constraints */
+PG_FUNCTION_INFO_V1(satisfies_hash_partition);
+
/*
* RelationBuildPartitionDesc
* Form rel's partition descriptor
@@ -163,6 +187,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -229,7 +256,33 @@ 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 = 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;
@@ -454,6 +507,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] = hbounds[i]->index;
+ remainder += modulus;
+ }
+
+ mapping[i] = i;
+ pfree(hbounds[i]);
+ }
+ pfree(hbounds);
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
boundinfo->has_null = found_null;
@@ -607,53 +696,81 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
+
+ /*
+ * 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++)
+ if (!(b1->datums[i][0] == b2->datums[i][0] &&
+ b1->datums[i][1] == b2->datums[i][1]))
+ return false;
- for (j = 0; j < key->partnatts; j++)
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -675,6 +792,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = true;
+ int pmod, /* Previous largest modulus */
+ nmod; /* 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 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)
+ {
+ nmod = DatumGetInt32(datums[0][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ else
+ {
+ pmod = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % pmod) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ nmod = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -897,6 +1097,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1144,6 +1349,105 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
/* Module-local functions */
/*
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder this function
+ * returns an expression Node for the partition table's CHECK constraint.
+ *
+ * For example, given a partition definition such 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 a
+ * subexpression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn(a), hash_fn(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn(a), hash_fn(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn(a), hash_fn(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn(a), hash_fn(b))
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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);
+
+ /* Form hash_fn(value) expression */
+ keyCol = (Node *) makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make1(keyCol),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+ }
+ else
+ {
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
+
+ args = lappend(args, keyCol);
+ }
+
+ fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+ BOOLOID,
+ args,
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+
+ return list_make1(fexpr);
+}
+
+/*
* get_qual_for_list
*
* Returns a list of expressions to use as a list partition's constraint.
@@ -1717,10 +2021,25 @@ 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);
+ uint32 rowHash = compute_hash_value(key->partsupfunc,
+ key->partnatts,
+ values, isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
+
/*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
*/
if (isnull[0])
cur_index = partdesc->boundinfo->null_index;
@@ -1731,7 +2050,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
if (cur_offset >= 0 && equal)
cur_index = partdesc->boundinfo->indexes[cur_offset];
else
@@ -1754,7 +2073,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
/*
* Offset returned is such that the bound at offset is
@@ -1796,6 +2115,26 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+ int v1 = h1->modulus;
+ int v2 = h2->modulus;
+
+ if (v1 < v2)
+ return -1;
+ if (v1 > v2)
+ return 1;
+ if (v1 == v2 && h1->remainder != h2->remainder)
+ return (h1->remainder > h2->remainder) ? 1 : -1;
+ return 0;
+}
+
+/*
* qsort_partition_list_value_cmp
*
* Compare two list partition bound datums
@@ -1972,6 +2311,25 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
switch (key->strategy)
{
+ case PARTITION_STRATEGY_HASH:
+ {
+ PartitionBoundSpec *spec = (PartitionBoundSpec *) probe;
+ int mod = DatumGetInt32(bound_datums[0]);
+
+ if (mod < spec->modulus)
+ cmpval = -1;
+ else if (mod > spec->modulus)
+ cmpval = 1;
+ else if (mod == spec->modulus)
+ {
+ int rem = DatumGetInt32(bound_datums[1]);
+
+ cmpval = rem == spec->remainder ? 0 :
+ (rem < spec->remainder ? -1 : 1);
+ }
+
+ break;
+ }
case PARTITION_STRATEGY_LIST:
cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
key->partcollation[0],
@@ -2055,3 +2413,81 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/* Mix hash bits */
+static uint32
+mix_hash_value(int nkeys, Datum *values, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /* rotate hash left 1 bit before mixing in the next column */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= DatumGetUInt32(values[i]);
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values,
+ bool *isnull)
+{
+ int i;
+ Datum hashValues[PARTITION_MAX_KEYS];
+
+ for (i = 0; i < nkeys; i++)
+ {
+ if (!isnull[i])
+ {
+ Assert(OidIsValid(partsupfunc[i].fn_oid));
+
+ hashValues[i] = FunctionCall1(&partsupfunc[i], values[i]);
+ }
+ }
+
+ /* Combine all hash values in hashValues array in single 32-bit hash value */
+ return mix_hash_value(nkeys, hashValues, isnull);
+}
+
+/*
+ * satisfies_hash_partition
+ *
+ * This is a SQL-callable function for use in hash partition constraints;
+ * 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 values[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /* key's hash values start from third argument of function. */
+ if (!PG_ARGISNULL(i + 2))
+ {
+ values[i] = PG_GETARG_DATUM(i + 2);
+ isnull[i] = false;
+ }
+ else
+ isnull[i] = true;
+ }
+
+ /* Combine all hash values in single 32-bit hash value */
+ rowHash = mix_hash_value(nkeys, values, isnull);
+
+ PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 49bb4d7..ebadeee 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -363,9 +363,9 @@ static ObjectAddress ATExecSetNotNull(AlteredTableInfo *tab, Relation rel,
static ObjectAddress ATExecColumnDefault(Relation rel, const char *colName,
Node *newDefault, LOCKMODE lockmode);
static ObjectAddress ATExecAddIdentity(Relation rel, const char *colName,
- Node *def, LOCKMODE lockmode);
+ Node *def, LOCKMODE lockmode);
static ObjectAddress ATExecSetIdentity(Relation rel, const char *colName,
- Node *def, LOCKMODE lockmode);
+ Node *def, LOCKMODE lockmode);
static ObjectAddress ATExecDropIdentity(Relation rel, const char *colName, bool missing_ok, LOCKMODE lockmode);
static void ATPrepSetStatistics(Relation rel, const char *colName,
Node *newValue, LOCKMODE lockmode);
@@ -459,7 +459,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, char strategy);
+ 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,
@@ -643,8 +643,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
descriptor->tdhasoid = (localHasOids || parentOidCount > 0);
/*
- * If a partitioned table doesn't have the system OID column, then none
- * of its partitions should have it.
+ * If a partitioned table doesn't have the system OID column, then none of
+ * its partitions should have it.
*/
if (stmt->partbound && parentOidCount == 0 && localHasOids)
ereport(ERROR,
@@ -1112,9 +1112,9 @@ RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid, Oid oldRelOid,
}
/*
- * Similarly, if we previously locked some other partition's heap, and
- * the name we're looking up no longer refers to that relation, release
- * the now-useless lock.
+ * Similarly, if we previously locked some other partition's heap, and the
+ * name we're looking up no longer refers to that relation, release the
+ * now-useless lock.
*/
if (relOid != oldRelOid && OidIsValid(state->partParentOid))
{
@@ -2219,8 +2219,8 @@ MergeAttributes(List *schema, List *supers, char relpersistence,
else
ereport(ERROR,
(errcode(ERRCODE_DUPLICATE_COLUMN),
- errmsg("column \"%s\" specified more than once",
- coldef->colname)));
+ errmsg("column \"%s\" specified more than once",
+ coldef->colname)));
}
prev = rest;
rest = next;
@@ -4541,7 +4541,7 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
values[ex->attnum - 1] = ExecEvalExpr(ex->exprstate,
econtext,
- &isnull[ex->attnum - 1]);
+ &isnull[ex->attnum - 1]);
}
/*
@@ -5589,12 +5589,12 @@ static void
ATPrepDropNotNull(Relation rel, bool recurse, bool recursing)
{
/*
- * If the parent is a partitioned table, like check constraints, we do
- * not support removing the NOT NULL while partitions exist.
+ * If the parent is a partitioned table, like check constraints, we do not
+ * support removing the NOT NULL while partitions exist.
*/
if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
- PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+ PartitionDesc partdesc = RelationGetPartitionDesc(rel);
Assert(partdesc != NULL);
if (partdesc->nparts > 0 && !recurse && !recursing)
@@ -5639,8 +5639,8 @@ ATExecDropNotNull(Relation rel, const char *colName, LOCKMODE lockmode)
if (get_attidentity(RelationGetRelid(rel), attnum))
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
- errmsg("column \"%s\" of relation \"%s\" is an identity column",
- colName, RelationGetRelationName(rel))));
+ errmsg("column \"%s\" of relation \"%s\" is an identity column",
+ colName, RelationGetRelationName(rel))));
/*
* Check that the attribute is not in a primary key
@@ -5768,7 +5768,7 @@ ATPrepSetNotNull(Relation rel, bool recurse, bool recursing)
*/
if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
- PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+ PartitionDesc partdesc = RelationGetPartitionDesc(rel);
if (partdesc && partdesc->nparts > 0 && !recurse && !recursing)
ereport(ERROR,
@@ -5867,8 +5867,8 @@ ATExecColumnDefault(Relation rel, const char *colName,
if (get_attidentity(RelationGetRelid(rel), attnum))
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
- errmsg("column \"%s\" of relation \"%s\" is an identity column",
- colName, RelationGetRelationName(rel)),
+ errmsg("column \"%s\" of relation \"%s\" is an identity column",
+ colName, RelationGetRelationName(rel)),
newDefault ? 0 : errhint("Use ALTER TABLE ... ALTER COLUMN ... DROP IDENTITY instead.")));
/*
@@ -5959,8 +5959,8 @@ ATExecAddIdentity(Relation rel, const char *colName,
if (attTup->atthasdef)
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("column \"%s\" of relation \"%s\" already has a default value",
- colName, RelationGetRelationName(rel))));
+ errmsg("column \"%s\" of relation \"%s\" already has a default value",
+ colName, RelationGetRelationName(rel))));
attTup->attidentity = cdef->identity;
CatalogTupleUpdate(attrelation, &tuple->t_self, tuple);
@@ -5986,7 +5986,7 @@ static ObjectAddress
ATExecSetIdentity(Relation rel, const char *colName, Node *def, LOCKMODE lockmode)
{
ListCell *option;
- DefElem *generatedEl = NULL;
+ DefElem *generatedEl = NULL;
HeapTuple tuple;
Form_pg_attribute attTup;
AttrNumber attnum;
@@ -5995,7 +5995,7 @@ ATExecSetIdentity(Relation rel, const char *colName, Node *def, LOCKMODE lockmod
foreach(option, castNode(List, def))
{
- DefElem *defel = lfirst_node(DefElem, option);
+ DefElem *defel = lfirst_node(DefElem, option);
if (strcmp(defel->defname, "generated") == 0)
{
@@ -6036,8 +6036,8 @@ ATExecSetIdentity(Relation rel, const char *colName, Node *def, LOCKMODE lockmod
if (!attTup->attidentity)
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("column \"%s\" of relation \"%s\" is not an identity column",
- colName, RelationGetRelationName(rel))));
+ errmsg("column \"%s\" of relation \"%s\" is not an identity column",
+ colName, RelationGetRelationName(rel))));
if (generatedEl)
{
@@ -9746,7 +9746,7 @@ RebuildConstraintComment(AlteredTableInfo *tab, int pass, Oid objid,
cmd = makeNode(CommentStmt);
cmd->objtype = OBJECT_TABCONSTRAINT;
cmd->object = (Node *) list_make3(makeString(get_namespace_name(RelationGetNamespace(rel))),
- makeString(RelationGetRelationName(rel)),
+ makeString(RelationGetRelationName(rel)),
makeString(conname));
cmd->comment = comment_str;
@@ -11119,7 +11119,7 @@ CreateInheritance(Relation child_rel, Relation parent_rel)
inhseqno + 1,
catalogRelation,
parent_rel->rd_rel->relkind ==
- RELKIND_PARTITIONED_TABLE);
+ RELKIND_PARTITIONED_TABLE);
/* Now we're done with pg_inherits */
heap_close(catalogRelation, RowExclusiveLock);
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13225,6 +13227,8 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
{
int attn;
ListCell *lc;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13367,25 +13371,38 @@ 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 a btree or hash opclass to use. Currently, we use only
+ * btree operators, which seems enough for list and range
+ * partitioning, and hash operators for hash partitioning.
*/
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 2d2a9d0..660441f 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 b5459cd..92aad4e 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 98f6768..808813b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 65c004c..18b69b4 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -580,7 +580,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
%type <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2653,8 +2654,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2679,6 +2718,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 882955b..61878a0 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -363,7 +363,7 @@ generateSerialExtraStmts(CreateStmtContext *cxt, ColumnDef *column,
char **snamespace_p, char **sname_p)
{
ListCell *option;
- DefElem *nameEl = NULL;
+ DefElem *nameEl = NULL;
Oid snamespaceid;
char *snamespace;
char *sname;
@@ -378,12 +378,12 @@ generateSerialExtraStmts(CreateStmtContext *cxt, ColumnDef *column,
* used by pg_dump. Else, generate a name.
*
* Although we use ChooseRelationName, it's not guaranteed that the
- * selected sequence name won't conflict; given sufficiently long
- * field names, two different serial columns in the same table could
- * be assigned the same sequence name, and we'd not notice since we
- * aren't creating the sequence quite yet. In practice this seems
- * quite unlikely to be a problem, especially since few people would
- * need two serial columns in one table.
+ * selected sequence name won't conflict; given sufficiently long field
+ * names, two different serial columns in the same table could be assigned
+ * the same sequence name, and we'd not notice since we aren't creating
+ * the sequence quite yet. In practice this seems quite unlikely to be a
+ * problem, especially since few people would need two serial columns in
+ * one table.
*/
foreach(option, seqoptions)
@@ -402,7 +402,8 @@ generateSerialExtraStmts(CreateStmtContext *cxt, ColumnDef *column,
if (nameEl)
{
- RangeVar *rv = makeRangeVarFromNameList(castNode(List, nameEl->arg));
+ RangeVar *rv = makeRangeVarFromNameList(castNode(List, nameEl->arg));
+
snamespace = rv->schemaname;
sname = rv->relname;
seqoptions = list_delete_ptr(seqoptions, nameEl);
@@ -429,14 +430,14 @@ generateSerialExtraStmts(CreateStmtContext *cxt, ColumnDef *column,
cxt->relation->relname, column->colname)));
/*
- * Build a CREATE SEQUENCE command to create the sequence object, and
- * add it to the list of things to be done before this CREATE/ALTER
- * TABLE.
+ * Build a CREATE SEQUENCE command to create the sequence object, and add
+ * it to the list of things to be done before this CREATE/ALTER TABLE.
*/
seqstmt = makeNode(CreateSeqStmt);
seqstmt->for_identity = for_identity;
seqstmt->sequence = makeRangeVar(snamespace, sname, -1);
seqstmt->options = seqoptions;
+
/*
* If a sequence data type was specified, add it to the options. Prepend
* to the list rather than append; in case a user supplied their own AS
@@ -448,11 +449,11 @@ generateSerialExtraStmts(CreateStmtContext *cxt, ColumnDef *column,
seqstmt->options);
/*
- * If this is ALTER ADD COLUMN, make sure the sequence will be owned
- * by the table's owner. The current user might be someone else
- * (perhaps a superuser, or someone who's only a member of the owning
- * role), but the SEQUENCE OWNED BY mechanisms will bleat unless table
- * and sequence have exactly the same owning role.
+ * If this is ALTER ADD COLUMN, make sure the sequence will be owned by
+ * the table's owner. The current user might be someone else (perhaps a
+ * superuser, or someone who's only a member of the owning role), but the
+ * SEQUENCE OWNED BY mechanisms will bleat unless table and sequence have
+ * exactly the same owning role.
*/
if (cxt->rel)
seqstmt->ownerId = cxt->rel->rd_rel->relowner;
@@ -462,9 +463,9 @@ generateSerialExtraStmts(CreateStmtContext *cxt, ColumnDef *column,
cxt->blist = lappend(cxt->blist, seqstmt);
/*
- * Build an ALTER SEQUENCE ... OWNED BY command to mark the sequence
- * as owned by this column, and add it to the list of things to be
- * done after this CREATE/ALTER TABLE.
+ * Build an ALTER SEQUENCE ... OWNED BY command to mark the sequence as
+ * owned by this column, and add it to the list of things to be done after
+ * this CREATE/ALTER TABLE.
*/
altseqstmt = makeNode(AlterSeqStmt);
altseqstmt->sequence = makeRangeVar(snamespace, sname, -1);
@@ -647,31 +648,31 @@ transformColumnDefinition(CreateStmtContext *cxt, ColumnDef *column)
break;
case CONSTR_IDENTITY:
- {
- Type ctype;
- Oid typeOid;
+ {
+ Type ctype;
+ Oid typeOid;
- ctype = typenameType(cxt->pstate, column->typeName, NULL);
- typeOid = HeapTupleGetOid(ctype);
- ReleaseSysCache(ctype);
+ ctype = typenameType(cxt->pstate, column->typeName, NULL);
+ typeOid = HeapTupleGetOid(ctype);
+ ReleaseSysCache(ctype);
- if (saw_identity)
- ereport(ERROR,
- (errcode(ERRCODE_SYNTAX_ERROR),
- errmsg("multiple identity specifications for column \"%s\" of table \"%s\"",
+ if (saw_identity)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("multiple identity specifications for column \"%s\" of table \"%s\"",
column->colname, cxt->relation->relname),
- parser_errposition(cxt->pstate,
- constraint->location)));
+ parser_errposition(cxt->pstate,
+ constraint->location)));
- generateSerialExtraStmts(cxt, column,
- typeOid, constraint->options, true,
- NULL, NULL);
+ generateSerialExtraStmts(cxt, column,
+ typeOid, constraint->options, true,
+ NULL, NULL);
- column->identity = constraint->generated_when;
- saw_identity = true;
- column->is_not_null = TRUE;
- break;
- }
+ column->identity = constraint->generated_when;
+ saw_identity = true;
+ column->is_not_null = TRUE;
+ break;
+ }
case CONSTR_CHECK:
cxt->ckconstraints = lappend(cxt->ckconstraints, constraint);
@@ -1036,7 +1037,7 @@ transformTableLikeClause(CreateStmtContext *cxt, TableLikeClause *table_like_cla
if (attribute->attidentity &&
(table_like_clause->options & CREATE_TABLE_LIKE_IDENTITY))
{
- Oid seq_relid;
+ Oid seq_relid;
List *seq_options;
/*
@@ -1067,7 +1068,7 @@ transformTableLikeClause(CreateStmtContext *cxt, TableLikeClause *table_like_cla
stmt->objtype = OBJECT_COLUMN;
stmt->object = (Node *) list_make3(makeString(cxt->relation->schemaname),
- makeString(cxt->relation->relname),
+ makeString(cxt->relation->relname),
makeString(def->colname));
stmt->comment = comment;
@@ -1132,7 +1133,7 @@ transformTableLikeClause(CreateStmtContext *cxt, TableLikeClause *table_like_cla
stmt->objtype = OBJECT_TABCONSTRAINT;
stmt->object = (Node *) list_make3(makeString(cxt->relation->schemaname),
- makeString(cxt->relation->relname),
+ makeString(cxt->relation->relname),
makeString(n->conname));
stmt->comment = comment;
@@ -2766,7 +2767,11 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
* change the data type of the sequence.
*/
attnum = get_attnum(relid, cmd->name);
- /* if attribute not found, something will error about it later */
+
+ /*
+ * if attribute not found, something will error about it
+ * later
+ */
if (attnum != InvalidAttrNumber && get_attidentity(relid, attnum))
{
Oid seq_relid = getOwnedSequence(relid, attnum);
@@ -2774,7 +2779,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
AlterSeqStmt *altseqstmt = makeNode(AlterSeqStmt);
altseqstmt->sequence = makeRangeVar(get_namespace_name(get_rel_namespace(seq_relid)),
- get_rel_name(seq_relid),
+ get_rel_name(seq_relid),
-1);
altseqstmt->options = list_make1(makeDefElem("as", (Node *) makeTypeNameFromOid(typeOid, -1), -1));
altseqstmt->for_identity = true;
@@ -2787,8 +2792,8 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
case AT_AddIdentity:
{
- Constraint *def = castNode(Constraint, cmd->def);
- ColumnDef *newdef = makeNode(ColumnDef);
+ Constraint *def = castNode(Constraint, cmd->def);
+ ColumnDef *newdef = makeNode(ColumnDef);
AttrNumber attnum;
newdef->colname = cmd->name;
@@ -2796,7 +2801,11 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
cmd->def = (Node *) newdef;
attnum = get_attnum(relid, cmd->name);
- /* if attribute not found, something will error about it later */
+
+ /*
+ * if attribute not found, something will error about it
+ * later
+ */
if (attnum != InvalidAttrNumber)
generateSerialExtraStmts(&cxt, newdef,
get_atttype(relid, attnum),
@@ -2825,7 +2834,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
*/
foreach(lc, castNode(List, cmd->def))
{
- DefElem *def = lfirst_node(DefElem, lc);
+ DefElem *def = lfirst_node(DefElem, lc);
if (strcmp(def->defname, "generated") == 0)
newdef = lappend(newdef, def);
@@ -2846,7 +2855,7 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
seqstmt = makeNode(AlterSeqStmt);
seq_relid = linitial_oid(seqlist);
seqstmt->sequence = makeRangeVar(get_namespace_name(get_rel_namespace(seq_relid)),
- get_rel_name(seq_relid), -1);
+ get_rel_name(seq_relid), -1);
seqstmt->options = newseqopts;
seqstmt->for_identity = true;
seqstmt->missing_ok = false;
@@ -2854,8 +2863,11 @@ transformAlterTableStmt(Oid relid, AlterTableStmt *stmt,
cxt.alist = lappend(cxt.alist, seqstmt);
}
}
- /* If column was not found or was not an identity column, we
- * just let the ALTER TABLE command error out later. */
+
+ /*
+ * If column was not found or was not an identity column,
+ * we just let the ALTER TABLE command error out later.
+ */
cmd->def = (Node *) newdef;
newcmds = lappend(newcmds, cmd);
@@ -3282,7 +3294,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be less than modulus")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than remainder")));
+ }
+ else if (strategy == PARTITION_STRATEGY_LIST)
{
ListCell *cell;
char *colname;
@@ -3392,8 +3427,8 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
else if (seen_unbounded)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
- errmsg("cannot specify finite value after UNBOUNDED"),
- parser_errposition(pstate, exprLocation((Node *) ldatum))));
+ errmsg("cannot specify finite value after UNBOUNDED"),
+ parser_errposition(pstate, exprLocation((Node *) ldatum))));
}
seen_unbounded = false;
foreach(cell1, spec->upperdatums)
@@ -3406,8 +3441,8 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
else if (seen_unbounded)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
- errmsg("cannot specify finite value after UNBOUNDED"),
- parser_errposition(pstate, exprLocation((Node *) rdatum))));
+ errmsg("cannot specify finite value after UNBOUNDED"),
+ parser_errposition(pstate, exprLocation((Node *) rdatum))));
}
i = j = 0;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index cbde1ff..9456d48 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1653,6 +1653,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)
appendStringInfo(&buf, "LIST");
@@ -8619,6 +8623,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 3bd5277..046385b 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2003,7 +2003,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 77d8ed5..d459f0b 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5468,6 +5468,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 2276" _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 46c23c2..ec9b082 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 15d4ce5..5aebb3d 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
@@ -479,6 +474,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +512,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,6 +543,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 (
@@ -526,6 +552,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +561,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +570,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -598,6 +632,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -684,6 +735,8 @@ Number of partitions: 3 (Use \d+ to list them.)
-- 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 6f34b1c..ecad15d 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 95035c5..361c240 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
@@ -451,6 +446,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 (
@@ -478,6 +475,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +498,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
@@ -492,6 +506,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +514,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +523,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +575,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -611,6 +643,8 @@ CREATE TABLE part_c_1_10 PARTITION OF part_c FOR VALUES FROM (1) TO (10);
-- 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 020854c..d361909 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Thu, May 11, 2017 at 9:32 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, May 3, 2017 at 6:39 PM, amul sul <sulamul@gmail.com> wrote:
On Thu, Apr 27, 2017 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:
I spent some time today looking at these patches. It seems like there
is some more work still needed here to produce something committable
regardless of which way we go, but I am inclined to think that Amul's
patch is a better basis for work going forward than Nagata-san's
patch. Here are some general comments on the two patches:Thanks for your time.
[...]
- Neither patch contains any documentation updates, which is bad.
Fixed in the attached version.
I have done an intial review of the patch and I have some comments. I
will continue the review
and testing and report the results soon-----
Patch need to be rebased----
if (key->strategy == PARTITION_STRATEGY_RANGE)
{
/* Disallow nulls in the range partition key of the tuple */
for (i = 0; i < key->partnatts; i++)
if (isnull[i])
ereport(ERROR,
(errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("range partition key of row contains null")));
}We need to add PARTITION_STRATEGY_HASH as well, we don't support NULL
for hash also, right?
----
We do.
RangeDatumContent **content;/* what's contained in each range bound datum?
* (see the above enum); NULL for list
* partitioned tables */This will be NULL for hash as well we need to change the comments.
-----
Fixed in previously posted patch(v3).
bool has_null; /* Is there a null-accepting partition? false
* for range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1Comments needs to be changed for these two members as well
----
Fixed in previously posted patch(v3).
+/* One bound of a hash partition */ +typedef struct PartitionHashBound +{ + int modulus; + int remainder; + int index; +} PartitionHashBound;It will good to add some comments to explain the structure members
I think we don't really need that, variable names are ample to explain
its purpose.
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 Wed, May 10, 2017 at 11:39 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, May 3, 2017 at 9:09 AM, amul sul <sulamul@gmail.com> wrote:
Fixed in the attached version.
+[ PARTITION BY { HASH | RANGE | LIST } ( { <replaceable
class="parameter">column_name</replaceable> | ( <replaceable
class="parameter">expression</replaceable> ) } [ COLLATE <replaceableIn the department of severe nitpicking, I would have expected this to
either use alphabetical order (HASH | LIST | RANGE) or to add the new
method at the end on the theory that we probably did the important
ones first (RANGE | LIST | HASH).
Importance is subjective, so may be we should arrange them in
alphabetical order, to keep the list in some order and be consistent
everywhere in the code and documentation.
More broadly, I wonder why we're
cramming this into the datums arrays instead of just adding another
field to PartitionBoundInfoData that is only used by hash
partitioning.
It would be good if we store datums corresponding to partition bounds
in the same place. So that we don't have to handle hash partition
specially in all the places where we handle partition bound datums. We
already do that for list and range partitions. May be we want to
continue doing so for hash as well. In my comments to Amul's latest
patch, I have described a possibility that partition_bounds_equal()
need not compare all entries in the datums array. It can just compare
greated modulus and the indexes array from given two partition bounds
to check whether they are equal. If that works well, we will probably
address your complaint about DatumIsEqual() in a different manner.
--
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, May 11, 2017 at 10:38 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
So, adding keycol IS NOT NULL (like we currently do for expressions) in
the implicit partition constraint would be more future-proof than
generating an actual catalogued NOT NULL constraint on the keycol? I now
tend to think it would be better. Directly inserting into a range
partition with a NULL value for a column currently generates a "null value
in column \"%s\" violates not-null constraint" instead of perhaps more
relevant "new row for relation \"%s\" violates partition constraint".
That said, we *do* document the fact that a NOT NULL constraint is added
on range key columns, but we might as well document instead that we don't
currently support routing tuples with NULL values in the partition key
through a range-partitioned table and so NULL values cause error.Can we still decide to do that instead?
I suggest you start a new thread on that topic.
--
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, May 12, 2017 at 6:08 PM, amul sul <sulamul@gmail.com> wrote:
Hi,
Please find the following updated patches attached:
0001-Cleanup.patch : Does some cleanup and code refactoring required
for hash partition patch. Otherwise, there will be unnecessary diff in
0002 patch
Thanks for splitting the patch.
+ if (isnull[0])
+ cur_index = partdesc->boundinfo->null_index;
This code assumes that null_index will be set to -1 when has_null is false. Per
RelationBuildPartitionDesc() this is true. But probably we should write this
code as
if (isnull[0])
{
if (partdesc->boundinfo->has_null)
cur_index = partdesc->boundinfo->null_index;
}
That way we are certain that when has_null is false, cur_index = -1 similar to
the original code.
Additional arguement to ComputePartitionAttrs() isn't used anywhere in this
patch, so may be this better be part of 0002. If we do this the only change
that will remain in patch is the refactoring of RelationBuildPartitionDesc(),
so we may consider merging it into 0002, unless we find that some more
refactoring is needed. But for now, having it as a separate patch helps.
Here's some more comments on 0002
+ * In the case of hash partitioning, datums is a 2-D array, stores modulus and
+ * remainder values at datums[x][0] and datums[x][1] respectively for each
+ * partition in the ascending order.
This comment about datums should appear in a paragraph of itself and may be
rephrased as in the attached patch. May be we could also add a line about
ndatums for hash partitioned tables as in the attached patch.
+ * (see the above enum); NULL for has and list
typo s/has/hash
+ if (key->strategy == PARTITION_STRATEGY_HASH)
+ {
+ ndatums = nparts;
+ hbounds = (PartitionHashBound **) palloc(nparts *
+
sizeof(PartitionHashBound *));
+ i = 0;
+ foreach (cell, boundspecs)
+ {
+ PartitionBoundSpec *spec = lfirst(cell);
+
[ clipped ]
+ hbounds[i]->index = i;
+ i++;
+ }
For list and range partitioned table we order the bounds so that two
partitioned tables have them in the same order irrespective of order in which
they are specified by the user or hence stored in the catalogs. The partitions
then get indexes according the order in which their bounds appear in ordered
arrays of bounds. Thus any two partitioned tables with same partition
specification always have same PartitionBoundInfoData. This helps in
partition-wise join to match partition bounds of two given tables. Above code
assigns the indexes to the partitions as they appear in the catalogs. This
means that two partitioned tables with same partition specification but
different order for partition bound specification will have different
PartitionBoundInfoData represenation.
If we do that, probably partition_bounds_equal() would reduce to just matching
indexes and the last element of datums array i.e. the greatest modulus datum.
If ordered datums array of two partitioned table do not match exactly, the
mismatch can be because missing datums or different datums. If it's a missing
datum it will change the greatest modulus or have corresponding entry in
indexes array as -1. If the entry differs it will cause mismatching indexes in
the index arrays.
+ * is not a factor of 15.
+ *
+ *
+ * Get greatest bound in array boundinfo->datums which is
An extra line here.
+ if (offset < 0)
+ {
+ nmod = DatumGetInt32(datums[0][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ else
+ {
+ pmod = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % pmod) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ nmod = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (nmod % spec->modulus) == 0;
+ }
+ }
May be name the variables as prev_mod(ulus) and next_mod(ulus) for better
readability.
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn(a), hash_fn(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn(a), hash_fn(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn(a), hash_fn(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn(a), hash_fn(b))
The description here may be read as if we are calling the same hash function
for both a and b, but that's not true. So, you may want to clarify that
in hash_fn(a) hash_fn means hash function specified for key a.
+ if (key->partattrs[i] != 0)
+ {
+ keyCol = (Node *) makeVar(1,
+ key->partattrs[i],
+ key->parttypid[i],
+ key->parttypmod[i],
+ key->parttypcoll[i],
+ 0);
+
+ /* Form hash_fn(value) expression */
+ keyCol = (Node *) makeFuncExpr(key->partsupfunc[i].fn_oid,
+ get_fn_expr_rettype(&key->partsupfunc[i]),
+ list_make1(keyCol),
+ InvalidOid,
+ InvalidOid,
+ COERCE_EXPLICIT_CALL);
+ }
+ else
+ {
+ keyCol = (Node *) copyObject(lfirst(partexprs_item));
+ partexprs_item = lnext(partexprs_item);
+ }
I think we should add FuncExpr for column Vars as well as expressions.
The logic to compare two bounds is duplicated in qsort_partition_hbound_cmp()
and partition_bound_cmp(). Should we separate it into a separate function
accepting moduli and remainders. That way in case we change it in future, we
have to change only one place.
I think we need more comments for compute_hash_value(), mix_hash_value() and
satisfies_hash_partition() as to what each of them accepts and what it
computes.
+ /* key's hash values start from third argument of function. */
+ if (!PG_ARGISNULL(i + 2))
+ {
+ values[i] = PG_GETARG_DATUM(i + 2);
+ isnull[i] = false;
+ }
+ else
+ isnull[i] = true;
You could write this as
isnull[i] = PG_ARGISNULL(i + 2);
if (isnull[i])
values[i] = PG_GETARG_DATUM(i + 2);
+ * Identify a btree or hash opclass to use. Currently, we use only
+ * btree operators, which seems enough for list and range partitioning,
+ * and hash operators for hash partitioning.
The wording, if not read carefully, might be read as "we use only btree
operators". I suggest we rephrase it as "Identify opclass to use. For
list and range
partitioning we use only btree operators, which seems enough for those. For
hash partitioning, we use hash operators." for clarity.
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
A search on WITH in gram.y shows that we do not handle WITH options in gram.y.
Usually they are handled at the transformation stage. Why is this an exception?
If you do that, we can have all the error handling in
transformPartitionBound().
+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 2276" _null_ _null_ _null_ _null_
_null_ satisfies_hash_partition _null_ _null_ _null_ ));
Why is third argument to this function ANY? Shouldn't it be INT4ARRAY (variadic
INT4)?
I am yet to review the testcases and thumb through all the places using
PARTITION_STRATEGY_RANGE/LIST to make sure that we are handling
PARTITION_STRATEGY_HASH in all those places.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
0002_additional_changes.patchtext/x-patch; charset=US-ASCII; name=0002_additional_changes.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 6bd4aa8..89c3b0a 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -59,9 +59,12 @@
* 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.
- * In the case of hash partitioning, datums is a 2-D array, stores modulus and
- * remainder values at datums[x][0] and datums[x][1] respectively for each
- * partition in the ascending order.
+ * In case of hash partitioning, ndatums 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.
@@ -87,12 +90,9 @@ typedef struct PartitionBoundInfoData
{
char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* For hash partitioned table, array of modulus
- * and remainder. For range and list partitioned
- * table, array of datum-tuples with
- * key->partnatts datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for has and list
+ * (see the above enum); NULL for hash and list
* partitioned tables */
int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
On Fri, May 12, 2017 at 6:08 PM, amul sul <sulamul@gmail.com> wrote:
Hi,
Please find the following updated patches attached:
I have done some testing with the new patch, most of the cases worked
as per the expectation except below
I expect the planner to select only "Seq Scan on t1" whereas it's
scanning both the partitions?
create table t (a int, b varchar) partition by hash(a);
create table t1 partition of t for values with (modulus 8, remainder 0);
create table t2 partition of t for values with (modulus 8, remainder 1);
postgres=# explain select * from t where a=8;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..51.75 rows=12 width=36)
-> Seq Scan on t1 (cost=0.00..25.88 rows=6 width=36)
Filter: (a = 8)
-> Seq Scan on t2 (cost=0.00..25.88 rows=6 width=36)
Filter: (a = 8)
(5 rows)
Some cosmetic comments.
-----------------------------------
+ RangeVar *rv = makeRangeVarFromNameList(castNode(List, nameEl->arg));
+
Useless Hunk.
/*
- * Build a CREATE SEQUENCE command to create the sequence object, and
- * add it to the list of things to be done before this CREATE/ALTER
- * TABLE.
+ * Build a CREATE SEQUENCE command to create the sequence object, and add
+ * it to the list of things to be done before this CREATE/ALTER TABLE.
*/
Seems like, in src/backend/parser/parse_utilcmd.c, you have changed
the existing code with
pgindent. I think it's not a good idea to mix pgindent changes with your patch.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 Fri, May 12, 2017 at 10:39 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Fri, May 12, 2017 at 6:08 PM, amul sul <sulamul@gmail.com> wrote:
Hi,
Please find the following updated patches attached:
0001-Cleanup.patch : Does some cleanup and code refactoring required
for hash partition patch. Otherwise, there will be unnecessary diff in
0002 patchThanks for splitting the patch.
+ if (isnull[0]) + cur_index = partdesc->boundinfo->null_index; This code assumes that null_index will be set to -1 when has_null is false. Per RelationBuildPartitionDesc() this is true. But probably we should write this code as if (isnull[0]) { if (partdesc->boundinfo->has_null) cur_index = partdesc->boundinfo->null_index; } That way we are certain that when has_null is false, cur_index = -1 similar to the original code.
Okay will add this. I still don't understood point of having has_null
variable, if no null accepting partition exists then null_index is
alway set to -1 in RelationBuildPartitionDesc. Anyway, let not change
the original code.
Additional arguement to ComputePartitionAttrs() isn't used anywhere in this
patch, so may be this better be part of 0002. If we do this the only change
that will remain in patch is the refactoring of RelationBuildPartitionDesc(),
so we may consider merging it into 0002, unless we find that some more
refactoring is needed. But for now, having it as a separate patch helps.
Okay.
Here's some more comments on 0002
+ * In the case of hash partitioning, datums is a 2-D array, stores modulus and + * remainder values at datums[x][0] and datums[x][1] respectively for each + * partition in the ascending order.This comment about datums should appear in a paragraph of itself and may be
rephrased as in the attached patch. May be we could also add a line about
ndatums for hash partitioned tables as in the attached patch.
Thanks, looks good to me; will include this.
[...]
+ if (key->strategy == PARTITION_STRATEGY_HASH) + { + ndatums = nparts; + hbounds = (PartitionHashBound **) palloc(nparts * + sizeof(PartitionHashBound *)); + i = 0; + foreach (cell, boundspecs) + { + PartitionBoundSpec *spec = lfirst(cell); + [ clipped ] + hbounds[i]->index = i; + i++; + } For list and range partitioned table we order the bounds so that two partitioned tables have them in the same order irrespective of order in which they are specified by the user or hence stored in the catalogs. The partitions then get indexes according the order in which their bounds appear in ordered arrays of bounds. Thus any two partitioned tables with same partition specification always have same PartitionBoundInfoData. This helps in partition-wise join to match partition bounds of two given tables. Above code assigns the indexes to the partitions as they appear in the catalogs. This means that two partitioned tables with same partition specification but different order for partition bound specification will have different PartitionBoundInfoData represenation.If we do that, probably partition_bounds_equal() would reduce to just matching
indexes and the last element of datums array i.e. the greatest modulus datum.
If ordered datums array of two partitioned table do not match exactly, the
mismatch can be because missing datums or different datums. If it's a missing
datum it will change the greatest modulus or have corresponding entry in
indexes array as -1. If the entry differs it will cause mismatching indexes in
the index arrays.
Make sense, will fix this.
[...]
+ if (offset < 0) + { + nmod = DatumGetInt32(datums[0][0]); + valid_bound = (nmod % spec->modulus) == 0; + } + else + { + pmod = DatumGetInt32(datums[offset][0]); + valid_bound = (spec->modulus % pmod) == 0; + + if (valid_bound && (offset + 1) < ndatums) + { + nmod = DatumGetInt32(datums[offset + 1][0]); + valid_bound = (nmod % spec->modulus) == 0; + } + } May be name the variables as prev_mod(ulus) and next_mod(ulus) for better readability.
Okay, will rename to prev_modulus and next_modulus resp.
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn(a), hash_fn(b)) + * for p_p2: satisfies_hash_partition(4, 2, hash_fn(a), hash_fn(b)) + * for p_p3: satisfies_hash_partition(8, 0, hash_fn(a), hash_fn(b)) + * for p_p4: satisfies_hash_partition(8, 4, hash_fn(a), hash_fn(b)) The description here may be read as if we are calling the same hash function for both a and b, but that's not true. So, you may want to clarify that in hash_fn(a) hash_fn means hash function specified for key a.
Okay.
+ if (key->partattrs[i] != 0) + { + keyCol = (Node *) makeVar(1, + key->partattrs[i], + key->parttypid[i], + key->parttypmod[i], + key->parttypcoll[i], + 0); + + /* Form hash_fn(value) expression */ + keyCol = (Node *) makeFuncExpr(key->partsupfunc[i].fn_oid, + get_fn_expr_rettype(&key->partsupfunc[i]), + list_make1(keyCol), + InvalidOid, + InvalidOid, + COERCE_EXPLICIT_CALL); + } + else + { + keyCol = (Node *) copyObject(lfirst(partexprs_item)); + partexprs_item = lnext(partexprs_item); + } I think we should add FuncExpr for column Vars as well as expressions.
Okay, will fix this.
The logic to compare two bounds is duplicated in qsort_partition_hbound_cmp()
and partition_bound_cmp(). Should we separate it into a separate function
accepting moduli and remainders. That way in case we change it in future, we
have to change only one place.
Okay.
I think we need more comments for compute_hash_value(), mix_hash_value() and
satisfies_hash_partition() as to what each of them accepts and what it
computes.+ /* key's hash values start from third argument of function. */ + if (!PG_ARGISNULL(i + 2)) + { + values[i] = PG_GETARG_DATUM(i + 2); + isnull[i] = false; + } + else + isnull[i] = true; You could write this as isnull[i] = PG_ARGISNULL(i + 2); if (isnull[i]) values[i] = PG_GETARG_DATUM(i + 2);
Okay.
+ * Identify a btree or hash opclass to use. Currently, we use only + * btree operators, which seems enough for list and range partitioning, + * and hash operators for hash partitioning.The wording, if not read carefully, might be read as "we use only btree
operators". I suggest we rephrase it as "Identify opclass to use. For
list and range
partitioning we use only btree operators, which seems enough for those. For
hash partitioning, we use hash operators." for clarity.
Okay
+ foreach (lc, $5) + { + DefElem *opt = (DefElem *) lfirst(lc); A search on WITH in gram.y shows that we do not handle WITH options in gram.y. Usually they are handled at the transformation stage. Why is this an exception? If you do that, we can have all the error handling in transformPartitionBound().
If so, ForValues need to return list for hash and PartitionBoundSpec
for other two; wouldn't that break code consistency? And such
validation is not new in gram.y see xmltable_column_el.
+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 2276" _null_ _null_ _null_ _null_
_null_ satisfies_hash_partition _null_ _null_ _null_ ));
Why is third argument to this function ANY? Shouldn't it be INT4ARRAY (variadic
INT4)?
Will use INT4ARRAY in next patch, but I am little sceptical of it. we
need an unsigned int32, but unfortunately there is not variadic uint32
support. How about INT8ARRAY?
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, May 13, 2017 at 12:11 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, May 12, 2017 at 6:08 PM, amul sul <sulamul@gmail.com> wrote:
Hi,
Please find the following updated patches attached:
I have done some testing with the new patch, most of the cases worked
as per the expectation except belowI expect the planner to select only "Seq Scan on t1" whereas it's
scanning both the partitions?create table t (a int, b varchar) partition by hash(a);
create table t1 partition of t for values with (modulus 8, remainder 0);
create table t2 partition of t for values with (modulus 8, remainder 1);postgres=# explain select * from t where a=8;
QUERY PLAN
----------------------------------------------------------
Append (cost=0.00..51.75 rows=12 width=36)
-> Seq Scan on t1 (cost=0.00..25.88 rows=6 width=36)
Filter: (a = 8)
-> Seq Scan on t2 (cost=0.00..25.88 rows=6 width=36)
Filter: (a = 8)
(5 rows)
You are correct. As of now constraint exclusion doesn't work on
partition constraint involves function call[1], and hash partition
constraint does have satisfies_hash_partition() function call.
Some cosmetic comments. ----------------------------------- + RangeVar *rv = makeRangeVarFromNameList(castNode(List, nameEl->arg)); +Useless Hunk.
/* - * Build a CREATE SEQUENCE command to create the sequence object, and - * add it to the list of things to be done before this CREATE/ALTER - * TABLE. + * Build a CREATE SEQUENCE command to create the sequence object, and add + * it to the list of things to be done before this CREATE/ALTER TABLE. */Seems like, in src/backend/parser/parse_utilcmd.c, you have changed
the existing code with
pgindent. I think it's not a good idea to mix pgindent changes with your patch.
Oops, my silly mistake, sorry about that. Fixed in attached version.
Regards,
Amul
1] /messages/by-id/CA+TgmoaE9NZ_RiqZQLp2aJXPO4E78QxkQYL-FR2zCDop96Ahdg@mail.gmail.com
Attachments:
0001-Cleanup_v2.patchapplication/octet-stream; name=0001-Cleanup_v2.patchDownload
From 8ef84a31e61271c007852ec100be63740b94a5b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sat, 13 May 2017 18:39:53 +0530
Subject: [PATCH 1/2] Cleanup_v2
Code refactoring required for hash partitioning patch v4
---
src/backend/catalog/partition.c | 87 ++++++++++++++++++++++-------------------
1 file changed, 47 insertions(+), 40 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 885c533..5566839 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -252,8 +252,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -334,8 +333,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1924,10 +1922,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,40 +1965,51 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partdesc->boundinfo->has_null)
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -2010,12 +2017,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-v4.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v4.patchDownload
From 9a0c76e03ed758ca4b0b058db86166ac0f84ffb6 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sun, 14 May 2017 13:17:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v4
v4:
Fixes w.r.t. Ashutosh's and Dilips's review comments.
v3:
Fixes w.r.t. Ashutosh's and Robert's review comments.
v2:
Regression test updated.
Documentation added.
Added tab complition for FOR VALUES WITH
v1:
Initial patch
---
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 57 ++-
src/backend/catalog/partition.c | 538 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 42 ++-
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 | 61 +++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 ++++
src/test/regress/expected/create_table.out | 63 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 ++++
src/test/regress/sql/create_table.sql | 44 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 +
22 files changed, 1113 insertions(+), 76 deletions(-)
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..1c57e52 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -1591,6 +1615,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
@@ -1641,6 +1675,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 5566839..f2433a9 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,23 @@
* 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.
+ * In case of hash partitioning, ndatums 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +87,17 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,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,
@@ -128,12 +145,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +168,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +196,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -237,7 +265,33 @@ 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 = 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;
@@ -462,6 +516,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] = hbounds[i]->index;
+ remainder += modulus;
+ }
+
+ mapping[i] = i;
+ pfree(hbounds[i]);
+ }
+ pfree(hbounds);
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
boundinfo->has_null = found_null;
@@ -615,53 +705,79 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
+
+ /*
+ * Compare greatest modulus of hash partition bound which
+ * is the last element of datums array.
+ */
+ if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
+ return false;
- for (j = 0; j < key->partnatts; j++)
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -683,6 +799,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -905,6 +1104,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1298,6 +1502,109 @@ make_partition_op_expr(PartitionKey key, int keynum,
}
/*
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder this function
+ * returns an expression Node for the partition table's CHECK constraint.
+ *
+ * For example, given a partition definition such 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 a
+ * subexpression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * hash_fn_1 and hash_fn_2 will be datatype-specific hash functions for
+ * column a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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
+ {
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2274,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
* A null partition key is only acceptable if null-accepting
@@ -2051,6 +2371,34 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2227,6 +2575,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],
@@ -2310,3 +2667,98 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Rotate hash left 1 bit before mixing in the next column. This
+ * prevents equal values in different keys from cancelling each other.
+ * */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 cdcb949..9f3438c 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13220,10 +13222,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13365,26 +13370,37 @@ 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 opclass to use. For list and range partitioning we use only
+ * btree operators, which seems enough for those. For hash partitioning,
+ * we use hash operators. */
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d13a6fc..76baeae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 c9a8c34..200ed7a 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 3d5b09a..08bca2b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 2822331..15324e5 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 882955b..716786c 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be less than modulus")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 c9bded0..74ce07d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -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)
appendStringInfo(&buf, "LIST");
@@ -8651,6 +8655,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ae37302..73fb3bf 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2022,7 +2022,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7434696..d7ad3ef 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 d396be3..6017910 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 bbf039c..b8cb1be 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
@@ -479,6 +474,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +512,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,6 +543,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 (
@@ -526,6 +552,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +561,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +570,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -599,6 +633,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -747,6 +798,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND ((abs(a
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 02429a3..b9ac539 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 766f35a..5a8ff99 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
@@ -451,6 +446,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 (
@@ -478,6 +475,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +498,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
@@ -492,6 +506,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +514,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +523,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +575,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -627,6 +659,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 db8967b..29a9704 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Wed, May 10, 2017 at 10:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, May 10, 2017 at 8:34 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:Hash partitioning will partition the data based on the hash value of the
partition key. Does that require collation? Should we throw an error/warning if
collation is specified in PARTITION BY clause?Collation is only relevant for ordering, not equality. Since hash
opclasses provide only equality, not ordering, it's not relevant here.
I'm not sure whether we should error out if it's specified or just
silently ignore it. Maybe an ERROR is a good idea? But not sure.
IMHO, we could simply have a WARNING, and ignore collation, thoughts?
Updated patches attached.
Regards,
Amul
Attachments:
0001-Cleanup_v2.patchapplication/octet-stream; name=0001-Cleanup_v2.patchDownload
From 8ef84a31e61271c007852ec100be63740b94a5b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sat, 13 May 2017 18:39:53 +0530
Subject: [PATCH 1/2] Cleanup_v2
Code refactoring required for hash partitioning patch v4
---
src/backend/catalog/partition.c | 87 ++++++++++++++++++++++-------------------
1 file changed, 47 insertions(+), 40 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 885c533..5566839 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -252,8 +252,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -334,8 +333,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1924,10 +1922,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,40 +1965,51 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partdesc->boundinfo->has_null)
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -2010,12 +2017,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-v5.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v5.patchDownload
From b5355cfdeef7a3e5992c68764119e03aec877409 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sun, 14 May 2017 13:17:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v5
v5:
Added code to throw warning if collation is specified for hash
partitioning
v4:
Fixes w.r.t. Ashutosh's and Dilips's review comments.
v3:
Fixes w.r.t. Ashutosh's and Robert's review comments.
v2:
Regression test updated.
Documentation added.
Added tab completion for FOR VALUES WITH
v1:
Initial patch
---
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 57 ++-
src/backend/catalog/partition.c | 538 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 58 +++-
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 | 61 +++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 ++++
src/test/regress/expected/create_table.out | 75 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 ++++
src/test/regress/sql/create_table.sql | 56 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 +
22 files changed, 1153 insertions(+), 76 deletions(-)
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..1c57e52 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -1591,6 +1615,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
@@ -1641,6 +1675,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 5566839..f2433a9 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,23 @@
* 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.
+ * In case of hash partitioning, ndatums 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +87,17 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,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,
@@ -128,12 +145,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +168,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +196,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -237,7 +265,33 @@ 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 = 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;
@@ -462,6 +516,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] = hbounds[i]->index;
+ remainder += modulus;
+ }
+
+ mapping[i] = i;
+ pfree(hbounds[i]);
+ }
+ pfree(hbounds);
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
boundinfo->has_null = found_null;
@@ -615,53 +705,79 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
+
+ /*
+ * Compare greatest modulus of hash partition bound which
+ * is the last element of datums array.
+ */
+ if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
+ return false;
- for (j = 0; j < key->partnatts; j++)
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -683,6 +799,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -905,6 +1104,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1298,6 +1502,109 @@ make_partition_op_expr(PartitionKey key, int keynum,
}
/*
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder this function
+ * returns an expression Node for the partition table's CHECK constraint.
+ *
+ * For example, given a partition definition such 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 a
+ * subexpression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * hash_fn_1 and hash_fn_2 will be datatype-specific hash functions for
+ * column a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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
+ {
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2274,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
* A null partition key is only acceptable if null-accepting
@@ -2051,6 +2371,34 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2227,6 +2575,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],
@@ -2310,3 +2667,98 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Rotate hash left 1 bit before mixing in the next column. This
+ * prevents equal values in different keys from cancelling each other.
+ * */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 cdcb949..ff508d9 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13197,6 +13199,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13209,6 +13212,21 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
assign_expr_collations(pstate, pelem->expr);
}
+ /*
+ * Collation is irrelevant for hash partition key, because hash
+ * operator classes provide only equality, not ordering.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ {
+ ereport(WARNING,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("collation for column \"%s\" is ignored", pelem->name),
+ parser_errposition(pstate, pelem->location)));
+
+ list_free(pelem->collation);
+ pelem->collation = NIL;
+ }
+
newspec->partParams = lappend(newspec->partParams, pelem);
}
@@ -13220,10 +13238,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13365,26 +13386,37 @@ 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 opclass to use. For list and range partitioning we use only
+ * btree operators, which seems enough for those. For hash partitioning,
+ * we use hash operators. */
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d13a6fc..76baeae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 c9a8c34..200ed7a 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 3d5b09a..08bca2b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 2822331..15324e5 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 882955b..716786c 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be less than modulus")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 c9bded0..74ce07d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -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)
appendStringInfo(&buf, "LIST");
@@ -8651,6 +8655,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ae37302..73fb3bf 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2022,7 +2022,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7434696..d7ad3ef 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 d396be3..6017910 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 bbf039c..b341f41 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
@@ -449,6 +444,18 @@ Partition key: RANGE (a oid_ops, plusone(b), c, d COLLATE "C")
Partition key: LIST ((a + 1))
DROP TABLE partitioned, partitioned2;
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant for hash partitioning. Will has warning if collation is
+-- specified.
+CREATE TABLE partitioned (
+ a int,
+ b int,
+ c text,
+ d text
+) PARTITION BY HASH (a oid_ops, plusone(b), c collate "default", d collate "C");
+WARNING: collation for column "c" is ignored
+WARNING: collation for column "d" is ignored
+DROP TABLE partitioned;
--
-- Partitions
--
@@ -479,6 +486,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +524,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,6 +555,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 (
@@ -526,6 +564,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +573,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +582,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -599,6 +645,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -747,6 +810,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND ((abs(a
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 02429a3..b9ac539 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 766f35a..d35eae8 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
@@ -430,6 +425,18 @@ CREATE TABLE fail () INHERITS (partitioned2);
DROP TABLE partitioned, partitioned2;
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant for hash partitioning. Will has warning if collation is
+-- specified.
+CREATE TABLE partitioned (
+ a int,
+ b int,
+ c text,
+ d text
+) PARTITION BY HASH (a oid_ops, plusone(b), c collate "default", d collate "C");
+
+DROP TABLE partitioned;
+
--
-- Partitions
--
@@ -451,6 +458,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 (
@@ -478,6 +487,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +510,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
@@ -492,6 +518,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +526,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +535,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +587,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -627,6 +671,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 db8967b..29a9704 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Mon, May 15, 2017 at 4:27 PM, amul sul <sulamul@gmail.com> wrote:
Updated patches attached.
While testing latest patch I found a strange behaviour.
test1:
postgres=# create table x (a int) partition by hash(a);
CREATE TABLE
postgres=# create table x1 partition of x for values with (modulus 4,
remainder 0);
CREATE TABLE
postgres=# create table x2 partition of x for values with (modulus 4,
remainder 1);
CREATE TABLE
postgres=# insert into x values(1);
2017-05-15 20:55:20.446 IST [28045] ERROR: no partition of relation
"x" found for row
2017-05-15 20:55:20.446 IST [28045] DETAIL: Partition key of the
failing row contains (a) = (1).
2017-05-15 20:55:20.446 IST [28045] STATEMENT: insert into x values(1);
ERROR: no partition of relation "x" found for row
DETAIL: Partition key of the failing row contains (a) = (1).
Test2:
postgres=# insert into x2 values(100); -- it should violates
partition constraint
INSERT 0 1
Seems like a bug or am I missing something completely?
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 Mon, May 15, 2017 at 6:57 AM, amul sul <sulamul@gmail.com> wrote:
Collation is only relevant for ordering, not equality. Since hash
opclasses provide only equality, not ordering, it's not relevant here.
I'm not sure whether we should error out if it's specified or just
silently ignore it. Maybe an ERROR is a good idea? But not sure.IMHO, we could simply have a WARNING, and ignore collation, thoughts?
Updated patches attached.
I think that WARNING is rarely a good compromise between ERROR and
nothing. I think we should just decide whether this is legal (and
then allow it without a WARNING) or not legal (and then ERROR).
Telling the user that it's allowed but we don't like it doesn't really
help much.
--
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, May 15, 2017 at 9:06 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
Test2:
postgres=# insert into x2 values(100); -- it should violates
partition constraint
INSERT 0 1Seems like a bug or am I missing something completely?
Sorry, my bad. It's modulus on the hashvalue, not the column.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 Mon, May 15, 2017 at 9:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, May 15, 2017 at 6:57 AM, amul sul <sulamul@gmail.com> wrote:
Collation is only relevant for ordering, not equality. Since hash
opclasses provide only equality, not ordering, it's not relevant here.
I'm not sure whether we should error out if it's specified or just
silently ignore it. Maybe an ERROR is a good idea? But not sure.IMHO, we could simply have a WARNING, and ignore collation, thoughts?
Updated patches attached.
I think that WARNING is rarely a good compromise between ERROR and
nothing. I think we should just decide whether this is legal (and
then allow it without a WARNING) or not legal (and then ERROR).
Telling the user that it's allowed but we don't like it doesn't really
help much.
+1. We should throw an error and add a line in documentation that
collation should not be specified for hash partitioned table.
--
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, May 15, 2017 at 9:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, May 15, 2017 at 6:57 AM, amul sul <sulamul@gmail.com> wrote:
Collation is only relevant for ordering, not equality. Since hash
opclasses provide only equality, not ordering, it's not relevant here.
I'm not sure whether we should error out if it's specified or just
silently ignore it. Maybe an ERROR is a good idea? But not sure.IMHO, we could simply have a WARNING, and ignore collation, thoughts?
Updated patches attached.
I think that WARNING is rarely a good compromise between ERROR and
nothing. I think we should just decide whether this is legal (and
then allow it without a WARNING) or not legal (and then ERROR).
Telling the user that it's allowed but we don't like it doesn't really
help much.
Understood, will throw an ERROR instead.
Thank you.
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, May 16, 2017 at 10:03 AM, amul sul <sulamul@gmail.com> wrote:
On Mon, May 15, 2017 at 9:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
Collation is only relevant for ordering, not equality.
While earlier, I thought the same, I am wondering whether this is
true. Don't different collations deem different strings equal e.g one
collation may deem 'aa' and 'AA' as same but other may not. Or is that
encoding problem being discussed in hash functions thread?
Sorry for the confusion.
--
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 Sun, May 14, 2017 at 12:30 PM, amul sul <sulamul@gmail.com> wrote:
On Fri, May 12, 2017 at 10:39 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:On Fri, May 12, 2017 at 6:08 PM, amul sul <sulamul@gmail.com> wrote:
Hi,
Please find the following updated patches attached:
0001-Cleanup.patch : Does some cleanup and code refactoring required
for hash partition patch. Otherwise, there will be unnecessary diff in
0002 patchThanks for splitting the patch.
+ if (isnull[0]) + cur_index = partdesc->boundinfo->null_index; This code assumes that null_index will be set to -1 when has_null is false. Per RelationBuildPartitionDesc() this is true. But probably we should write this code as if (isnull[0]) { if (partdesc->boundinfo->has_null) cur_index = partdesc->boundinfo->null_index; } That way we are certain that when has_null is false, cur_index = -1 similar to the original code.Okay will add this.
Thanks.
I still don't understood point of having has_null
variable, if no null accepting partition exists then null_index is
alway set to -1 in RelationBuildPartitionDesc. Anyway, let not change
the original code.
I agree. has_null might have been folded as null_index == -1. But
that's not the problem of this patch.
0001 looks good to me now.
[...]
+ if (key->strategy == PARTITION_STRATEGY_HASH) + { + ndatums = nparts; + hbounds = (PartitionHashBound **) palloc(nparts * + sizeof(PartitionHashBound *)); + i = 0; + foreach (cell, boundspecs) + { + PartitionBoundSpec *spec = lfirst(cell); + [ clipped ] + hbounds[i]->index = i; + i++; + } For list and range partitioned table we order the bounds so that two partitioned tables have them in the same order irrespective of order in which they are specified by the user or hence stored in the catalogs. The partitions then get indexes according the order in which their bounds appear in ordered arrays of bounds. Thus any two partitioned tables with same partition specification always have same PartitionBoundInfoData. This helps in partition-wise join to match partition bounds of two given tables. Above code assigns the indexes to the partitions as they appear in the catalogs. This means that two partitioned tables with same partition specification but different order for partition bound specification will have different PartitionBoundInfoData represenation.If we do that, probably partition_bounds_equal() would reduce to just matching
indexes and the last element of datums array i.e. the greatest modulus datum.
If ordered datums array of two partitioned table do not match exactly, the
mismatch can be because missing datums or different datums. If it's a missing
datum it will change the greatest modulus or have corresponding entry in
indexes array as -1. If the entry differs it will cause mismatching indexes in
the index arrays.Make sense, will fix this.
I don't see this being addressed in the patches attached in the reply to Dilip.
+ if (key->partattrs[i] != 0) + { + keyCol = (Node *) makeVar(1, + key->partattrs[i], + key->parttypid[i], + key->parttypmod[i], + key->parttypcoll[i], + 0); + + /* Form hash_fn(value) expression */ + keyCol = (Node *) makeFuncExpr(key->partsupfunc[i].fn_oid, + get_fn_expr_rettype(&key->partsupfunc[i]), + list_make1(keyCol), + InvalidOid, + InvalidOid, + COERCE_EXPLICIT_CALL); + } + else + { + keyCol = (Node *) copyObject(lfirst(partexprs_item)); + partexprs_item = lnext(partexprs_item); + } I think we should add FuncExpr for column Vars as well as expressions.Okay, will fix this.
Here, please add a check similar to get_quals_for_range()
1840 if (partexprs_item == NULL)
1841 elog(ERROR, "wrong number of partition key expressions");
I think we need more comments for compute_hash_value(), mix_hash_value() and
satisfies_hash_partition() as to what each of them accepts and what it
computes.+ /* key's hash values start from third argument of function. */ + if (!PG_ARGISNULL(i + 2)) + { + values[i] = PG_GETARG_DATUM(i + 2); + isnull[i] = false; + } + else + isnull[i] = true; You could write this as isnull[i] = PG_ARGISNULL(i + 2); if (isnull[i]) values[i] = PG_GETARG_DATUM(i + 2);Okay.
If we have used this technique somewhere else in PG code, please
mention that function/place.
/*
* Rotate hash left 1 bit before mixing in the next column. This
* prevents equal values in different keys from cancelling each other.
*/
+ foreach (lc, $5) + { + DefElem *opt = (DefElem *) lfirst(lc); A search on WITH in gram.y shows that we do not handle WITH options in gram.y. Usually they are handled at the transformation stage. Why is this an exception? If you do that, we can have all the error handling in transformPartitionBound().If so, ForValues need to return list for hash and PartitionBoundSpec
for other two; wouldn't that break code consistency? And such
validation is not new in gram.y see xmltable_column_el.
Thanks for pointing that out. Ok, then may be leave it in gram.y. But
may be we should move the error handling in transform 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 2276" _null_ _null_ _null_ _null_
_null_ satisfies_hash_partition _null_ _null_ _null_ ));
Why is third argument to this function ANY? Shouldn't it be INT4ARRAY (variadic
INT4)?Will use INT4ARRAY in next patch, but I am little sceptical of it. we
need an unsigned int32, but unfortunately there is not variadic uint32
support. How about INT8ARRAY?
Hmm, I think as long as the binary representation of given unsigned
integer doesn't change in the function call, we could cast an INT32
datums into unsigned int32, so spending extra 4 bytes per partition
key doesn't look like worth the effort.
A related question is, all hash functions have return type as
"integer" but internally they return uint32. Why not to do the same
for this function as well?
--
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
Hi,
Here's patch with some cosmetic fixes to 0002, to be applied on top of 0002.
On Tue, May 16, 2017 at 1:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Sun, May 14, 2017 at 12:30 PM, amul sul <sulamul@gmail.com> wrote:
On Fri, May 12, 2017 at 10:39 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:On Fri, May 12, 2017 at 6:08 PM, amul sul <sulamul@gmail.com> wrote:
Hi,
Please find the following updated patches attached:
0001-Cleanup.patch : Does some cleanup and code refactoring required
for hash partition patch. Otherwise, there will be unnecessary diff in
0002 patchThanks for splitting the patch.
+ if (isnull[0]) + cur_index = partdesc->boundinfo->null_index; This code assumes that null_index will be set to -1 when has_null is false. Per RelationBuildPartitionDesc() this is true. But probably we should write this code as if (isnull[0]) { if (partdesc->boundinfo->has_null) cur_index = partdesc->boundinfo->null_index; } That way we are certain that when has_null is false, cur_index = -1 similar to the original code.Okay will add this.
Thanks.
I still don't understood point of having has_null
variable, if no null accepting partition exists then null_index is
alway set to -1 in RelationBuildPartitionDesc. Anyway, let not change
the original code.I agree. has_null might have been folded as null_index == -1. But
that's not the problem of this patch.0001 looks good to me now.
[...]
+ if (key->strategy == PARTITION_STRATEGY_HASH) + { + ndatums = nparts; + hbounds = (PartitionHashBound **) palloc(nparts * + sizeof(PartitionHashBound *)); + i = 0; + foreach (cell, boundspecs) + { + PartitionBoundSpec *spec = lfirst(cell); + [ clipped ] + hbounds[i]->index = i; + i++; + } For list and range partitioned table we order the bounds so that two partitioned tables have them in the same order irrespective of order in which they are specified by the user or hence stored in the catalogs. The partitions then get indexes according the order in which their bounds appear in ordered arrays of bounds. Thus any two partitioned tables with same partition specification always have same PartitionBoundInfoData. This helps in partition-wise join to match partition bounds of two given tables. Above code assigns the indexes to the partitions as they appear in the catalogs. This means that two partitioned tables with same partition specification but different order for partition bound specification will have different PartitionBoundInfoData represenation.If we do that, probably partition_bounds_equal() would reduce to just matching
indexes and the last element of datums array i.e. the greatest modulus datum.
If ordered datums array of two partitioned table do not match exactly, the
mismatch can be because missing datums or different datums. If it's a missing
datum it will change the greatest modulus or have corresponding entry in
indexes array as -1. If the entry differs it will cause mismatching indexes in
the index arrays.Make sense, will fix this.
I don't see this being addressed in the patches attached in the reply to Dilip.
+ if (key->partattrs[i] != 0) + { + keyCol = (Node *) makeVar(1, + key->partattrs[i], + key->parttypid[i], + key->parttypmod[i], + key->parttypcoll[i], + 0); + + /* Form hash_fn(value) expression */ + keyCol = (Node *) makeFuncExpr(key->partsupfunc[i].fn_oid, + get_fn_expr_rettype(&key->partsupfunc[i]), + list_make1(keyCol), + InvalidOid, + InvalidOid, + COERCE_EXPLICIT_CALL); + } + else + { + keyCol = (Node *) copyObject(lfirst(partexprs_item)); + partexprs_item = lnext(partexprs_item); + } I think we should add FuncExpr for column Vars as well as expressions.Okay, will fix this.
Here, please add a check similar to get_quals_for_range()
1840 if (partexprs_item == NULL)
1841 elog(ERROR, "wrong number of partition key expressions");I think we need more comments for compute_hash_value(), mix_hash_value() and
satisfies_hash_partition() as to what each of them accepts and what it
computes.+ /* key's hash values start from third argument of function. */ + if (!PG_ARGISNULL(i + 2)) + { + values[i] = PG_GETARG_DATUM(i + 2); + isnull[i] = false; + } + else + isnull[i] = true; You could write this as isnull[i] = PG_ARGISNULL(i + 2); if (isnull[i]) values[i] = PG_GETARG_DATUM(i + 2);Okay.
If we have used this technique somewhere else in PG code, please
mention that function/place.
/*
* Rotate hash left 1 bit before mixing in the next column. This
* prevents equal values in different keys from cancelling each other.
*/+ foreach (lc, $5) + { + DefElem *opt = (DefElem *) lfirst(lc); A search on WITH in gram.y shows that we do not handle WITH options in gram.y. Usually they are handled at the transformation stage. Why is this an exception? If you do that, we can have all the error handling in transformPartitionBound().If so, ForValues need to return list for hash and PartitionBoundSpec
for other two; wouldn't that break code consistency? And such
validation is not new in gram.y see xmltable_column_el.Thanks for pointing that out. Ok, then may be leave it in gram.y. But
may be we should move the error handling in transform 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 2276" _null_ _null_ _null_ _null_
_null_ satisfies_hash_partition _null_ _null_ _null_ ));
Why is third argument to this function ANY? Shouldn't it be INT4ARRAY (variadic
INT4)?Will use INT4ARRAY in next patch, but I am little sceptical of it. we
need an unsigned int32, but unfortunately there is not variadic uint32
support. How about INT8ARRAY?Hmm, I think as long as the binary representation of given unsigned
integer doesn't change in the function call, we could cast an INT32
datums into unsigned int32, so spending extra 4 bytes per partition
key doesn't look like worth the effort.A related question is, all hash functions have return type as
"integer" but internally they return uint32. Why not to do the same
for this function as well?--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
0002-cosmetic_fixes.patchtext/x-patch; charset=US-ASCII; name=0002-cosmetic_fixes.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 987bb73..e9b09dd 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,7 +58,8 @@
* 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.
- * In case of hash partitioning, ndatums be same as the number of partitions.
+ * 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.
@@ -1504,12 +1505,12 @@ make_partition_op_expr(PartitionKey key, int keynum,
/*
* get_qual_for_hash
*
- * Given a list of partition columns, modulus and remainder this function
- * returns an expression Node for the partition table's CHECK constraint.
+ * Given a list of partition columns, modulus and remainder corresponding to a
+ * partition, this function returns CHECK constraint expression Node for that
+ * partition.
*
- * For example, given a partition definition such as:
- * CREATE TABLE simple_hash (a int, b char(10))
- * PARTITION BY HASH (a, b);
+ * 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);
@@ -1520,16 +1521,16 @@ make_partition_op_expr(PartitionKey key, int keynum,
* 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 a
- * subexpression:
+ * 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(a), hash_fn_2(b))
* for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
* for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
* for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
*
- * hash_fn_1 and hash_fn_2 will be datatype-specific hash functions for
- * column a and b respectively.
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
*/
static List *
get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
@@ -1568,7 +1569,6 @@ get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
/* Left operand */
if (key->partattrs[i] != 0)
- {
keyCol = (Node *) makeVar(1,
key->partattrs[i],
key->parttypid[i],
@@ -1576,7 +1576,6 @@ get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
key->parttypcoll[i],
0);
- }
else
{
keyCol = (Node *) copyObject(lfirst(partexprs_item));
@@ -2684,7 +2683,7 @@ mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
/*
* Rotate hash left 1 bit before mixing in the next column. This
* prevents equal values in different keys from cancelling each other.
- * */
+ */
rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
if (!isnull[i])
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 46344ee..8f8bbcd 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -13231,8 +13231,9 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
}
/*
- * Collation is irrelevant for hash partition key, because hash
- * operator classes provide only equality, not ordering.
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not equality is
+ * irrelevant for hash partitioning.
*/
if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
{
@@ -13404,9 +13405,10 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
partcollation[attn] = attcollation;
- /* Identify opclass to use. For list and range partitioning we use only
- * btree operators, which seems enough for those. For hash partitioning,
- * we use hash operators. */
+ /*
+ * Identify opclass to use. For list and range partitioning we use only
+ * btree operator class, which seems enough for those. For hash partitioning,
+ * we use hash operator class. */
if (strategy == PARTITION_STRATEGY_HASH)
{
am_method = "hash";
On Tue, May 16, 2017 at 1:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
[...]
+ if (key->strategy == PARTITION_STRATEGY_HASH) + { + ndatums = nparts; + hbounds = (PartitionHashBound **) palloc(nparts * + sizeof(PartitionHashBound *)); + i = 0; + foreach (cell, boundspecs) + { + PartitionBoundSpec *spec = lfirst(cell); + [ clipped ] + hbounds[i]->index = i; + i++; + } For list and range partitioned table we order the bounds so that two partitioned tables have them in the same order irrespective of order in which they are specified by the user or hence stored in the catalogs. The partitions then get indexes according the order in which their bounds appear in ordered arrays of bounds. Thus any two partitioned tables with same partition specification always have same PartitionBoundInfoData. This helps in partition-wise join to match partition bounds of two given tables. Above code assigns the indexes to the partitions as they appear in the catalogs. This means that two partitioned tables with same partition specification but different order for partition bound specification will have different PartitionBoundInfoData represenation.If we do that, probably partition_bounds_equal() would reduce to just matching
indexes and the last element of datums array i.e. the greatest modulus datum.
If ordered datums array of two partitioned table do not match exactly, the
mismatch can be because missing datums or different datums. If it's a missing
datum it will change the greatest modulus or have corresponding entry in
indexes array as -1. If the entry differs it will cause mismatching indexes in
the index arrays.Make sense, will fix this.
I don't see this being addressed in the patches attached in the reply to Dilip.
Fixed in the attached version.
+ if (key->partattrs[i] != 0) + { + keyCol = (Node *) makeVar(1, + key->partattrs[i], + key->parttypid[i], + key->parttypmod[i], + key->parttypcoll[i], + 0); + + /* Form hash_fn(value) expression */ + keyCol = (Node *) makeFuncExpr(key->partsupfunc[i].fn_oid, + get_fn_expr_rettype(&key->partsupfunc[i]), + list_make1(keyCol), + InvalidOid, + InvalidOid, + COERCE_EXPLICIT_CALL); + } + else + { + keyCol = (Node *) copyObject(lfirst(partexprs_item)); + partexprs_item = lnext(partexprs_item); + } I think we should add FuncExpr for column Vars as well as expressions.Okay, will fix this.
Here, please add a check similar to get_quals_for_range()
1840 if (partexprs_item == NULL)
1841 elog(ERROR, "wrong number of partition key expressions");
Fixed in the attached version.
I think we need more comments for compute_hash_value(), mix_hash_value() and
satisfies_hash_partition() as to what each of them accepts and what it
computes.+ /* key's hash values start from third argument of function. */ + if (!PG_ARGISNULL(i + 2)) + { + values[i] = PG_GETARG_DATUM(i + 2); + isnull[i] = false; + } + else + isnull[i] = true; You could write this as isnull[i] = PG_ARGISNULL(i + 2); if (isnull[i]) values[i] = PG_GETARG_DATUM(i + 2);Okay.
If we have used this technique somewhere else in PG code, please
mention that function/place.
/*
* Rotate hash left 1 bit before mixing in the next column. This
* prevents equal values in different keys from cancelling each other.
*/
Fixed in the attached version.
+ foreach (lc, $5) + { + DefElem *opt = (DefElem *) lfirst(lc); A search on WITH in gram.y shows that we do not handle WITH options in gram.y. Usually they are handled at the transformation stage. Why is this an exception? If you do that, we can have all the error handling in transformPartitionBound().If so, ForValues need to return list for hash and PartitionBoundSpec
for other two; wouldn't that break code consistency? And such
validation is not new in gram.y see xmltable_column_el.Thanks for pointing that out. Ok, then may be leave it in gram.y. But
may be we should move the error handling in transform function.
IMO, let it be there for readability. It will be easier to understand
why do we have set -1 for modulus and remainder.
+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 2276" _null_ _null_ _null_ _null_
_null_ satisfies_hash_partition _null_ _null_ _null_ ));
Why is third argument to this function ANY? Shouldn't it be INT4ARRAY (variadic
INT4)?Will use INT4ARRAY in next patch, but I am little sceptical of it. we
need an unsigned int32, but unfortunately there is not variadic uint32
support. How about INT8ARRAY?Hmm, I think as long as the binary representation of given unsigned
integer doesn't change in the function call, we could cast an INT32
datums into unsigned int32, so spending extra 4 bytes per partition
key doesn't look like worth the effort.A related question is, all hash functions have return type as
"integer" but internally they return uint32. Why not to do the same
for this function as well?
I see. IIUC, there is no harm to use INT4ARRAY, thanks for explanation.
Regards,
Amul
Attachments:
0001-Cleanup_v2.patchapplication/octet-stream; name=0001-Cleanup_v2.patchDownload
From 8ef84a31e61271c007852ec100be63740b94a5b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sat, 13 May 2017 18:39:53 +0530
Subject: [PATCH 1/2] Cleanup_v2
Code refactoring required for hash partitioning patch v4
---
src/backend/catalog/partition.c | 87 ++++++++++++++++++++++-------------------
1 file changed, 47 insertions(+), 40 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 885c533..5566839 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -252,8 +252,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -334,8 +333,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1924,10 +1922,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,40 +1965,51 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partdesc->boundinfo->has_null)
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -2010,12 +2017,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-v6.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v6.patchDownload
From f638f32578b9be1f92fae2a4ba555560de16cb57 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sun, 14 May 2017 13:17:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v6
v6:
Fixed Ashutosh's review comments.
Documentation update.
v5:
Added code to throw warning if collation is specified for hash
partitioning
v4:
Fixes w.r.t. Ashutosh's and Dilips's review comments.
v3:
Fixes w.r.t. Ashutosh's and Robert's review comments.
v2:
Regression test updated.
Documentation added.
Added tab completion for FOR VALUES WITH
v1:
Initial patch
---
doc/src/sgml/ddl.sgml | 37 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 71 +++-
src/backend/catalog/partition.c | 540 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 55 ++-
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 | 61 +++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 ++++
src/test/regress/expected/create_table.out | 70 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 ++++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 +
23 files changed, 1184 insertions(+), 85 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..63269e1 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying remainder and modulus 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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -2902,8 +2916,9 @@ VALUES ('Albany', NULL, NULL, 'NY');
<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.
+ partitioning methods include hash, range and list, where each partition is
+ assigned a modulus and remainder of keys, a range of keys and a list of
+ keys, respectively.
</para>
<para>
@@ -3296,6 +3311,14 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
not the partitioned table.
</para>
</listitem>
+
+ <listitem>
+ <para>
+ An <command>UPDATE</> that causes a row to move from one partition to
+ another fails, because the new value of the row fails to satisfy the
+ implicit partition constraint of the original partition.
+ </para>
+ </listitem>
</itemizedlist>
</para>
</sect3>
@@ -3327,11 +3350,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..8e67e6a 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,17 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. List and range partitioning uses only btree operator class.
+ Hash partitioning uses only hash operator class. If no operator class is
+ specified when creating a partitioned table, the default operator class
+ for the datatype will be used. If there is none, an error will be
+ reported.
+ </para>
+
+ <para>
+ Since hash partitiong operator class, provide only equality, not ordering,
+ collation is not relevant in hash partition key column. An error will be
+ reported if collation is specified.
</para>
<para>
@@ -1591,6 +1623,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
@@ -1641,6 +1683,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 5566839..ec1b294 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +88,17 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,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,
@@ -128,12 +146,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +169,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -237,7 +266,33 @@ 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 = 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;
@@ -462,6 +517,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[i] = hbounds[i]->index;
+ pfree(hbounds[i]);
+ }
+ pfree(hbounds);
+ break;
+ }
+
case PARTITION_STRATEGY_LIST:
{
boundinfo->has_null = found_null;
@@ -615,53 +706,79 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
+
+ /*
+ * Compare greatest modulus of hash partition bound which
+ * is the last element of datums array.
+ */
+ if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
+ return false;
- for (j = 0; j < key->partnatts; j++)
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -683,6 +800,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -905,6 +1105,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1298,6 +1503,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2275,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
* A null partition key is only acceptable if null-accepting
@@ -2051,6 +2372,34 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2227,6 +2576,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],
@@ -2310,3 +2668,99 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 cdcb949..44ae028 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13197,6 +13199,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13209,6 +13212,19 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
assign_expr_collations(pstate, pelem->expr);
}
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not equality is
+ * irrelevant for hash partitioning.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot use collation for hash partition key column \"%s\"",
+ pelem->name),
+ parser_errposition(pstate, pelem->location)));
+
+
newspec->partParams = lappend(newspec->partParams, pelem);
}
@@ -13220,10 +13236,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13366,25 +13385,37 @@ 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 opclass to use. For list and range partitioning we use only
+ * btree operator class, which seems enough for those. For hash partitioning,
+ * we use hash operator class. */
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d13a6fc..76baeae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 c9a8c34..200ed7a 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 3d5b09a..08bca2b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 2822331..15324e5 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 882955b..716786c 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be less than modulus")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 c9bded0..74ce07d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -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)
appendStringInfo(&buf, "LIST");
@@ -8651,6 +8655,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ae37302..73fb3bf 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2022,7 +2022,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7434696..d7ad3ef 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 d396be3..6017910 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 bbf039c..a316f67 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -310,6 +310,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
ERROR: column "a" appears more than once in partition key
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+ERROR: cannot use collation for hash partition key column "a"
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -340,11 +347,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
@@ -479,6 +481,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +519,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,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 (
@@ -526,6 +559,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +568,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +577,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -599,6 +640,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -747,6 +805,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND ((abs(a
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 02429a3..b9ac539 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 766f35a..834cba2 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -320,6 +320,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -350,11 +357,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
@@ -451,6 +453,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 (
@@ -478,6 +482,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +505,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
@@ -492,6 +513,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +521,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +530,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +582,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -627,6 +666,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 db8967b..29a9704 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Tue, May 16, 2017 at 1:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Hi,
Here's patch with some cosmetic fixes to 0002, to be applied on top of 0002.
Thank you, included in v6 patch.
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, May 16, 2017 at 3:30 PM, amul sul <sulamul@gmail.com> wrote:
On Tue, May 16, 2017 at 1:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
[...]+ if (key->strategy == PARTITION_STRATEGY_HASH) + { + ndatums = nparts; + hbounds = (PartitionHashBound **) palloc(nparts * + sizeof(PartitionHashBound *)); + i = 0; + foreach (cell, boundspecs) + { + PartitionBoundSpec *spec = lfirst(cell); + [ clipped ] + hbounds[i]->index = i; + i++; + } For list and range partitioned table we order the bounds so that two partitioned tables have them in the same order irrespective of order in which they are specified by the user or hence stored in the catalogs. The partitions then get indexes according the order in which their bounds appear in ordered arrays of bounds. Thus any two partitioned tables with same partition specification always have same PartitionBoundInfoData. This helps in partition-wise join to match partition bounds of two given tables. Above code assigns the indexes to the partitions as they appear in the catalogs. This means that two partitioned tables with same partition specification but different order for partition bound specification will have different PartitionBoundInfoData represenation.If we do that, probably partition_bounds_equal() would reduce to just matching
indexes and the last element of datums array i.e. the greatest modulus datum.
If ordered datums array of two partitioned table do not match exactly, the
mismatch can be because missing datums or different datums. If it's a missing
datum it will change the greatest modulus or have corresponding entry in
indexes array as -1. If the entry differs it will cause mismatching indexes in
the index arrays.Make sense, will fix this.
I don't see this being addressed in the patches attached in the reply to Dilip.
Fixed in the attached version.
v6 patch has bug in partition oid mapping and indexing, fixed in the
attached version.
Now partition oids will be arranged in the ascending order of hash
partition bound (i.e. modulus and remainder sorting order)
Regards,
Amul
Attachments:
0001-Cleanup_v2.patchapplication/octet-stream; name=0001-Cleanup_v2.patchDownload
From 8ef84a31e61271c007852ec100be63740b94a5b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sat, 13 May 2017 18:39:53 +0530
Subject: [PATCH 1/2] Cleanup_v2
Code refactoring required for hash partitioning patch v4
---
src/backend/catalog/partition.c | 87 ++++++++++++++++++++++-------------------
1 file changed, 47 insertions(+), 40 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 885c533..5566839 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -252,8 +252,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -334,8 +333,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1924,10 +1922,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,40 +1965,51 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partdesc->boundinfo->has_null)
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -2010,12 +2017,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-v7.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v7.patchDownload
From 16618deb328e7f797474a9697ac82dfa0f8ee14e Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sun, 14 May 2017 13:17:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v7
v7:
Fixed bug in oids mapping, now relation oid will be in asceding order
of hash bound i.e. modulus and remainder.
v6:
Fixed Ashutosh's review comments.
Documentation update.
v5:
Added code to throw warning if collation is specified for hash
partitioning
v4:
Fixes w.r.t. Ashutosh's and Dilips's review comments.
v3:
Fixes w.r.t. Ashutosh's and Robert's review comments.
v2:
Regression test updated.
Documentation added.
Added tab completion for FOR VALUES WITH
v1:
Initial patch
rebase
---
doc/src/sgml/ddl.sgml | 37 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 71 +++-
src/backend/catalog/partition.c | 540 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 55 ++-
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 | 61 +++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 ++++
src/test/regress/expected/create_table.out | 70 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 ++++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 +
23 files changed, 1184 insertions(+), 85 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..63269e1 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<varlistentry>
+ <term>Hash Partitioning</term>
+
+ <listitem>
+ <para>
+ The table is partitioned by specifying remainder and modulus 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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -2902,8 +2916,9 @@ VALUES ('Albany', NULL, NULL, 'NY');
<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.
+ partitioning methods include hash, range and list, where each partition is
+ assigned a modulus and remainder of keys, a range of keys and a list of
+ keys, respectively.
</para>
<para>
@@ -3296,6 +3311,14 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
not the partitioned table.
</para>
</listitem>
+
+ <listitem>
+ <para>
+ An <command>UPDATE</> that causes a row to move from one partition to
+ another fails, because the new value of the row fails to satisfy the
+ implicit partition constraint of the original partition.
+ </para>
+ </listitem>
</itemizedlist>
</para>
</sect3>
@@ -3327,11 +3350,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..8e67e6a 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,17 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. List and range partitioning uses only btree operator class.
+ Hash partitioning uses only hash operator class. If no operator class is
+ specified when creating a partitioned table, the default operator class
+ for the datatype will be used. If there is none, an error will be
+ reported.
+ </para>
+
+ <para>
+ Since hash partitiong operator class, provide only equality, not ordering,
+ collation is not relevant in hash partition key column. An error will be
+ reported if collation is specified.
</para>
<para>
@@ -1591,6 +1623,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
@@ -1641,6 +1683,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 5566839..e080f05 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +88,17 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,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,
@@ -128,12 +146,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +169,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -237,7 +266,33 @@ 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 = 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;
@@ -462,6 +517,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->has_null = found_null;
@@ -615,53 +706,79 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
+
+ /*
+ * Compare greatest modulus of hash partition bound which
+ * is the last element of datums array.
+ */
+ if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
+ return false;
- for (j = 0; j < key->partnatts; j++)
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -683,6 +800,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -905,6 +1105,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1298,6 +1503,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2275,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
* A null partition key is only acceptable if null-accepting
@@ -2051,6 +2372,34 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2227,6 +2576,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],
@@ -2310,3 +2668,99 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 cdcb949..44ae028 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13197,6 +13199,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13209,6 +13212,19 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
assign_expr_collations(pstate, pelem->expr);
}
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not equality is
+ * irrelevant for hash partitioning.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot use collation for hash partition key column \"%s\"",
+ pelem->name),
+ parser_errposition(pstate, pelem->location)));
+
+
newspec->partParams = lappend(newspec->partParams, pelem);
}
@@ -13220,10 +13236,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13366,25 +13385,37 @@ 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 opclass to use. For list and range partitioning we use only
+ * btree operator class, which seems enough for those. For hash partitioning,
+ * we use hash operator class. */
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d13a6fc..76baeae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 c9a8c34..200ed7a 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 3d5b09a..08bca2b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 2822331..15324e5 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 882955b..716786c 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be less than modulus")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 c9bded0..74ce07d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -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)
appendStringInfo(&buf, "LIST");
@@ -8651,6 +8655,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ae37302..73fb3bf 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2022,7 +2022,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7434696..d7ad3ef 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 d396be3..6017910 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 bbf039c..a316f67 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -310,6 +310,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
ERROR: column "a" appears more than once in partition key
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+ERROR: cannot use collation for hash partition key column "a"
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -340,11 +347,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
@@ -479,6 +481,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +519,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,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 (
@@ -526,6 +559,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +568,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +577,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -599,6 +640,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -747,6 +805,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND ((abs(a
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 02429a3..b9ac539 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 766f35a..834cba2 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -320,6 +320,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -350,11 +357,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
@@ -451,6 +453,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 (
@@ -478,6 +482,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +505,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
@@ -492,6 +513,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +521,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +530,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +582,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -627,6 +666,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 db8967b..29a9704 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On 5/15/17 23:45, Ashutosh Bapat wrote:
+1. We should throw an error and add a line in documentation that
collation should not be specified for hash partitioned table.
Why is it even allowed in the parser then?
--
Peter Eisentraut http://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 5/16/17 03:19, Ashutosh Bapat wrote:
On Tue, May 16, 2017 at 10:03 AM, amul sul <sulamul@gmail.com> wrote:
On Mon, May 15, 2017 at 9:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
Collation is only relevant for ordering, not equality.
While earlier, I thought the same, I am wondering whether this is
true. Don't different collations deem different strings equal e.g one
collation may deem 'aa' and 'AA' as same but other may not. Or is that
encoding problem being discussed in hash functions thread?
The collations we currently support don't do that, unless someone made a
custom one. However, we might want to support that in the future.
Also, text/varchar comparisons always use strcmp() as a tie-breaker.
Again, this might be something to review at some point.
But you currently have the citext type that would indeed consider 'aa'
and 'AA' equal. But citext also has a hash function in the hash
operator class that handles that. So you could look into using that
approach.
--
Peter Eisentraut http://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 Tue, May 16, 2017 at 4:22 PM, amul sul <sulamul@gmail.com> wrote:
v6 patch has bug in partition oid mapping and indexing, fixed in the
attached version.Now partition oids will be arranged in the ascending order of hash
partition bound (i.e. modulus and remainder sorting order)
Thanks for the update patch. I have some more comments.
------------
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be less than modulus")));
I think this error message is not correct, you might want to change it
to "hash partition remainder must be non-negative integer"
-------
+ The table is partitioned by specifying remainder and modulus for each
+ partition. Each partition holds rows for which the hash value of
Wouldn't it be better to say "modulus and remainder" instead of
"remainder and modulus" then it will be consistent?
-------
+ An <command>UPDATE</> that causes a row to move from one partition to
+ another fails, because
fails, because -> fails because
-------
Wouldn't it be a good idea to document how to increase the number of
hash partitions, I think we can document it somewhere with an example,
something like Robert explained upthread?
create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);
You can detach foo1, create two new partitions with modulus 4 and
remainders 0 and 2, and move the data over from the old partition
I think it will be good information for a user to have? or it's
already documented and I missed it?
--------
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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, May 16, 2017 at 3:19 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
While earlier, I thought the same, I am wondering whether this is
true. Don't different collations deem different strings equal e.g one
collation may deem 'aa' and 'AA' as same but other may not.
No, that's not allowed. This has been discussed many times on this
mailing list. See varstr_cmp(), which you will notice refuses to
return 0 unless the strings are bytewise identical.
Or is that
encoding problem being discussed in hash functions thread?
No, that's something else entirely.
--
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, May 16, 2017 at 10:00 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, May 16, 2017 at 4:22 PM, amul sul <sulamul@gmail.com> wrote:
v6 patch has bug in partition oid mapping and indexing, fixed in the
attached version.Now partition oids will be arranged in the ascending order of hash
partition bound (i.e. modulus and remainder sorting order)Thanks for the update patch. I have some more comments.
------------ + if (spec->remainder < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("hash partition remainder must be less than modulus")));I think this error message is not correct, you might want to change it
to "hash partition remainder must be non-negative integer"
Fixed in the attached version; used "hash partition remainder must be
greater than or equal to 0" instead.
-------
+ The table is partitioned by specifying remainder and modulus for each + partition. Each partition holds rows for which the hash value ofWouldn't it be better to say "modulus and remainder" instead of
"remainder and modulus" then it will be consistent?
You are correct, fixed in the attached version.
------- + An <command>UPDATE</> that causes a row to move from one partition to + another fails, becausefails, because -> fails because
This hunk is no longer exists in the attached patch, that was mistaken
copied, sorry about that.
-------
Wouldn't it be a good idea to document how to increase the number of
hash partitions, I think we can document it somewhere with an example,
something like Robert explained upthread?create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);You can detach foo1, create two new partitions with modulus 4 and
remainders 0 and 2, and move the data over from the old partitionI think it will be good information for a user to have? or it's
already documented and I missed it?
I think, we should, but not sure about it.
Regards,
Amul
Attachments:
0001-Cleanup_v2.patchapplication/octet-stream; name=0001-Cleanup_v2.patchDownload
From 8ef84a31e61271c007852ec100be63740b94a5b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sat, 13 May 2017 18:39:53 +0530
Subject: [PATCH 1/2] Cleanup_v2
Code refactoring required for hash partitioning patch v4
---
src/backend/catalog/partition.c | 87 ++++++++++++++++++++++-------------------
1 file changed, 47 insertions(+), 40 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 885c533..5566839 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -252,8 +252,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -334,8 +333,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1924,10 +1922,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,40 +1965,51 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partdesc->boundinfo->has_null)
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -2010,12 +2017,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-v8.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v8.patchDownload
From e9113bce769663f560dd3e959b2e133b15b52aee Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sun, 14 May 2017 13:17:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v8
v8:
Fixed review comment by Dilip and couple of cosmetic fixes suggested by
Ashutosh.
v7:
Fixed bug in oids mapping, now relation oid will be in asceding order
of hash bound i.e. modulus and remainder.
v6:
Fixed Ashutosh's review comments.
Documentation update.
v5:
Added code to throw warning if collation is specified for hash
partitioning
v4:
Fixes w.r.t. Ashutosh's and Dilips's review comments.
v3:
Fixes w.r.t. Ashutosh's and Robert's review comments.
v2:
Regression test updated.
Documentation added.
Added tab completion for FOR VALUES WITH
v1:
Initial patch
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 71 +++-
src/backend/catalog/partition.c | 540 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 54 ++-
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 | 61 +++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 ++++
src/test/regress/expected/create_table.out | 70 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 ++++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 +
23 files changed, 1176 insertions(+), 84 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..6efe1e0 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -2902,8 +2916,9 @@ VALUES ('Albany', NULL, NULL, 'NY');
<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.
+ partitioning methods include hash, range and list, where each partition is
+ assigned a modulus and remainder of keys, a range of keys and a list of
+ keys, respectively.
</para>
<para>
@@ -3327,11 +3342,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..8e67e6a 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,17 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. List and range partitioning uses only btree operator class.
+ Hash partitioning uses only hash operator class. If no operator class is
+ specified when creating a partitioned table, the default operator class
+ for the datatype will be used. If there is none, an error will be
+ reported.
+ </para>
+
+ <para>
+ Since hash partitiong operator class, provide only equality, not ordering,
+ collation is not relevant in hash partition key column. An error will be
+ reported if collation is specified.
</para>
<para>
@@ -1591,6 +1623,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
@@ -1641,6 +1683,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 5566839..e080f05 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +88,17 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,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,
@@ -128,12 +146,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +169,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -237,7 +266,33 @@ 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 = 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;
@@ -462,6 +517,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->has_null = found_null;
@@ -615,53 +706,79 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
+
+ /*
+ * Compare greatest modulus of hash partition bound which
+ * is the last element of datums array.
+ */
+ if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
+ return false;
- for (j = 0; j < key->partnatts; j++)
+ /* Compare indexes */
+ 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;
+ }
+ else
+ {
+ for (i = 0; i < b1->ndatums; i++)
{
- /* For range partitions, the bounds might not be finite. */
- if (b1->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -683,6 +800,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -905,6 +1105,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1298,6 +1503,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2275,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
* A null partition key is only acceptable if null-accepting
@@ -2051,6 +2372,34 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2227,6 +2576,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],
@@ -2310,3 +2668,99 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 cdcb949..a71c2f3 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13197,6 +13199,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13209,6 +13212,19 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
assign_expr_collations(pstate, pelem->expr);
}
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not for equality, is
+ * irrelevant for hash partitioning.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot use collation for hash partition key column \"%s\"",
+ pelem->name),
+ parser_errposition(pstate, pelem->location)));
+
+
newspec->partParams = lappend(newspec->partParams, pelem);
}
@@ -13220,10 +13236,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13366,25 +13385,38 @@ 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 opclass to use. For list and range partitioning we use only
+ * btree operator class, which seems enough for those. For hash partitioning,
+ * we use hash operator class.
*/
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d13a6fc..76baeae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 c9a8c34..200ed7a 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 3d5b09a..08bca2b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 2822331..15324e5 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 882955b..cd95444 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be greater than or equal to 0")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 c9bded0..74ce07d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -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)
appendStringInfo(&buf, "LIST");
@@ -8651,6 +8655,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ae37302..73fb3bf 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2022,7 +2022,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7434696..d7ad3ef 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 d396be3..6017910 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 bbf039c..a316f67 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -310,6 +310,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
ERROR: column "a" appears more than once in partition key
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+ERROR: cannot use collation for hash partition key column "a"
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -340,11 +347,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
@@ -479,6 +481,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +519,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,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 (
@@ -526,6 +559,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +568,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +577,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -599,6 +640,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -747,6 +805,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND ((abs(a
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 02429a3..b9ac539 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 766f35a..834cba2 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -320,6 +320,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -350,11 +357,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
@@ -451,6 +453,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 (
@@ -478,6 +482,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +505,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
@@ -492,6 +513,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +521,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +530,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +582,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -627,6 +666,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 db8967b..29a9704 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
On Tue, May 16, 2017 at 6:50 PM, Peter Eisentraut
<peter.eisentraut@2ndquadrant.com> wrote:
On 5/15/17 23:45, Ashutosh Bapat wrote:
+1. We should throw an error and add a line in documentation that
collation should not be specified for hash partitioned table.Why is it even allowed in the parser then?
That grammar is common to all the partitioning strategies. It looks
like it's easy to handle collation for hash partitions in
transformation than in grammar. But, if we could handle it in grammar,
I don't have any objection to it.
--
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
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> writes:
On Tue, May 16, 2017 at 6:50 PM, Peter Eisentraut
<peter.eisentraut@2ndquadrant.com> wrote:On 5/15/17 23:45, Ashutosh Bapat wrote:
+1. We should throw an error and add a line in documentation that
collation should not be specified for hash partitioned table.
Why is it even allowed in the parser then?
That grammar is common to all the partitioning strategies. It looks
like it's easy to handle collation for hash partitions in
transformation than in grammar. But, if we could handle it in grammar,
I don't have any objection to it.
If you disallow something in the grammar, the error message is unlikely to
be better than "syntax error". That's not very desirable.
regards, tom lane
--
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, May 17, 2017 at 9:38 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote:
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> writes:
On Tue, May 16, 2017 at 6:50 PM, Peter Eisentraut
<peter.eisentraut@2ndquadrant.com> wrote:On 5/15/17 23:45, Ashutosh Bapat wrote:
+1. We should throw an error and add a line in documentation that
collation should not be specified for hash partitioned table.Why is it even allowed in the parser then?
That grammar is common to all the partitioning strategies. It looks
like it's easy to handle collation for hash partitions in
transformation than in grammar. But, if we could handle it in grammar,
I don't have any objection to it.If you disallow something in the grammar, the error message is unlikely to
be better than "syntax error". That's not very desirable.
Right +1.
--
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 Wed, May 17, 2017 at 12:04 AM, amul sul <sulamul@gmail.com> wrote:
On Tue, May 16, 2017 at 10:00 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, May 16, 2017 at 4:22 PM, amul sul <sulamul@gmail.com> wrote:
v6 patch has bug in partition oid mapping and indexing, fixed in the
attached version.Now partition oids will be arranged in the ascending order of hash
partition bound (i.e. modulus and remainder sorting order)Thanks for the update patch. I have some more comments.
------------ + if (spec->remainder < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("hash partition remainder must be less than modulus")));I think this error message is not correct, you might want to change it
to "hash partition remainder must be non-negative integer"Fixed in the attached version; used "hash partition remainder must be
greater than or equal to 0" instead.
I would suggest "non-zero positive", since that's what we are using in
the documentation.
-------
+ The table is partitioned by specifying remainder and modulus for each + partition. Each partition holds rows for which the hash value ofWouldn't it be better to say "modulus and remainder" instead of
"remainder and modulus" then it will be consistent?You are correct, fixed in the attached version.
------- + An <command>UPDATE</> that causes a row to move from one partition to + another fails, becausefails, because -> fails because
This hunk is no longer exists in the attached patch, that was mistaken
copied, sorry about that.-------
Wouldn't it be a good idea to document how to increase the number of
hash partitions, I think we can document it somewhere with an example,
something like Robert explained upthread?create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);You can detach foo1, create two new partitions with modulus 4 and
remainders 0 and 2, and move the data over from the old partitionI think it will be good information for a user to have? or it's
already documented and I missed it?
This is already part of documentation contained in the patch.
Here are some more comments
@@ -3296,6 +3311,14 @@ ALTER TABLE measurement ATTACH PARTITION
measurement_y2008m02
not the partitioned table.
</para>
</listitem>
+
+ <listitem>
+ <para>
+ An <command>UPDATE</> that causes a row to move from one partition to
+ another fails, because the new value of the row fails to satisfy the
+ implicit partition constraint of the original partition.
+ </para>
+ </listitem>
</itemizedlist>
</para>
</sect3>
The description in this chunk is applicable to all the kinds of partitioning.
Why should it be part of a patch implementing hash partitioning?
+ Declarative partitioning only supports hash, 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.)
Looks like the line width is less than 80 characters.
In partition_bounds_equal(), please add comments explaining why is it safe to
check just the indexes? May be we should add code under assertion to make sure
that the datums are equal as well. The comment could be something
like, "If two partitioned tables have different greatest moduli, their
partition schemes don't match. If they have same greatest moduli, and
all remainders have different indexes, they all have same modulus
specified and the partitions are ordered by remainders, thus indexes
array will be an identity i.e. index[i] = i. If the partition
corresponding to a given remainder exists, it will have same index
entry for both partitioned tables or if it's missing it will be -1.
Thus if indexes array matches, corresponding datums array matches. If
there are multiple remainders corresponding to a given partition,
their partitions are ordered by the lowest of the remainders, thus if
indexes array matches, both of the tables have same indexes arrays, in
both the tables remainders corresponding to multiple partitions all
have same indexes and thus same modulus. Thus again if the indexes are
same, datums are same.".
In the same function
if (key->strategy == PARTITION_STRATEGY_HASH)
{
int greatest_modulus;
/*
* Compare greatest modulus of hash partition bound which
* is the last element of datums array.
*/
if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
return false;
/* Compare indexes */
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;
}
if we return true from where this block ends, we will save one indenation level
for rest of the code and also FWIW extra diffs in this patch because of this
indentation change.
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not equality is
+ * irrelevant for hash partitioning.
+ */
A comma is missing after "equality", and may be we need "for" before
"equality".
* Collation, which is relevant for ordering and not equality, is
+ * we use hash operator class. */
*/ should be on new line.
--
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 Wed, May 17, 2017 at 11:11 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Wed, May 17, 2017 at 12:04 AM, amul sul <sulamul@gmail.com> wrote:
On Tue, May 16, 2017 at 10:00 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, May 16, 2017 at 4:22 PM, amul sul <sulamul@gmail.com> wrote:
v6 patch has bug in partition oid mapping and indexing, fixed in the
attached version.Now partition oids will be arranged in the ascending order of hash
partition bound (i.e. modulus and remainder sorting order)Thanks for the update patch. I have some more comments.
------------ + if (spec->remainder < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("hash partition remainder must be less than modulus")));I think this error message is not correct, you might want to change it
to "hash partition remainder must be non-negative integer"Fixed in the attached version; used "hash partition remainder must be
greater than or equal to 0" instead.I would suggest "non-zero positive", since that's what we are using in
the documentation.
Understood, Fixed in the attached version.
-------
+ The table is partitioned by specifying remainder and modulus for each + partition. Each partition holds rows for which the hash value ofWouldn't it be better to say "modulus and remainder" instead of
"remainder and modulus" then it will be consistent?You are correct, fixed in the attached version.
------- + An <command>UPDATE</> that causes a row to move from one partition to + another fails, becausefails, because -> fails because
This hunk is no longer exists in the attached patch, that was mistaken
copied, sorry about that.-------
Wouldn't it be a good idea to document how to increase the number of
hash partitions, I think we can document it somewhere with an example,
something like Robert explained upthread?create table foo (a integer, b text) partition by hash (a);
create table foo1 partition of foo with (modulus 2, remainder 0);
create table foo2 partition of foo with (modulus 2, remainder 1);You can detach foo1, create two new partitions with modulus 4 and
remainders 0 and 2, and move the data over from the old partitionI think it will be good information for a user to have? or it's
already documented and I missed it?This is already part of documentation contained in the patch.
Here are some more comments @@ -3296,6 +3311,14 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02 not the partitioned table. </para> </listitem> + + <listitem> + <para> + An <command>UPDATE</> that causes a row to move from one partition to + another fails, because the new value of the row fails to satisfy the + implicit partition constraint of the original partition. + </para> + </listitem> </itemizedlist> </para> </sect3> The description in this chunk is applicable to all the kinds of partitioning. Why should it be part of a patch implementing hash partitioning?
This was already addressed in the previous patch(v8).
+ Declarative partitioning only supports hash, 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.) Looks like the line width is less than 80 characters.
Fixed in the attached version.
In partition_bounds_equal(), please add comments explaining why is it safe to
check just the indexes? May be we should add code under assertion to make sure
that the datums are equal as well.
Added assert in the attached version.
The comment could be something
like, "If two partitioned tables have different greatest moduli, their
partition schemes don't match. If they have same greatest moduli, and
all remainders have different indexes, they all have same modulus
specified and the partitions are ordered by remainders, thus indexes
array will be an identity i.e. index[i] = i. If the partition
corresponding to a given remainder exists, it will have same index
entry for both partitioned tables or if it's missing it will be -1.
Thus if indexes array matches, corresponding datums array matches. If
there are multiple remainders corresponding to a given partition,
their partitions are ordered by the lowest of the remainders, thus if
indexes array matches, both of the tables have same indexes arrays, in
both the tables remainders corresponding to multiple partitions all
have same indexes and thus same modulus. Thus again if the indexes are
same, datums are same.".
Thanks, added with minor modification.
In the same function
if (key->strategy == PARTITION_STRATEGY_HASH)
{
int greatest_modulus;/*
* Compare greatest modulus of hash partition bound which
* is the last element of datums array.
*/
if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
return false;/* Compare indexes */
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;
}
if we return true from where this block ends, we will save one indenation level
for rest of the code and also FWIW extra diffs in this patch because of this
indentation change.
I still do believe having this code in the IF - ELSE block will be
better for longterm, rather having code clutter to avoid diff that
unpleasant for now.
+ /* + * Hash operator classes provide only equality, not ordering. + * Collation, which is relevant for ordering and not equality is + * irrelevant for hash partitioning. + */ A comma is missing after "equality", and may be we need "for" before "equality". * Collation, which is relevant for ordering and not equality, is+ * we use hash operator class. */
*/ should be on new line.
Fixed.
Regards,
Amul
Attachments:
0002-hash-partitioning_another_design-v9.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v9.patchDownload
From 91ac36fe2b182f7507aaa3b71b95fccb90c48493 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sun, 14 May 2017 13:17:13 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v9
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 71 +++-
src/backend/catalog/partition.c | 572 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 54 ++-
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 | 61 ++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 58 +++
src/test/regress/expected/create_table.out | 70 +++-
src/test/regress/expected/insert.out | 90 +++++
src/test/regress/expected/update.out | 21 ++
src/test/regress/sql/alter_table.sql | 59 +++
src/test/regress/sql/create_table.sql | 51 ++-
src/test/regress/sql/insert.sql | 70 ++++
src/test/regress/sql/update.sql | 19 +
23 files changed, 1208 insertions(+), 84 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..8159730 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -2902,8 +2916,9 @@ VALUES ('Albany', NULL, NULL, 'NY');
<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.
+ partitioning methods include hash, range and list, where each partition is
+ assigned a modulus and remainder of keys, a range of keys and a list of
+ keys, respectively.
</para>
<para>
@@ -3327,11 +3342,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 484f818..8e67e6a 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,17 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. List and range partitioning uses only btree operator class.
+ Hash partitioning uses only hash operator class. If no operator class is
+ specified when creating a partitioned table, the default operator class
+ for the datatype will be used. If there is none, an error will be
+ reported.
+ </para>
+
+ <para>
+ Since hash partitiong operator class, provide only equality, not ordering,
+ collation is not relevant in hash partition key column. An error will be
+ reported if collation is specified.
</para>
<para>
@@ -1591,6 +1623,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
@@ -1641,6 +1683,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 5566839..4f4cb56 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,20 +88,17 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); NULL for hash and
+ * list partitioned tables */
+ int *indexes; /* Partition indexes */
bool has_null; /* Is there a null-accepting partition? false
- * for range partitioned tables */
+ * for hash and range partitioned tables */
int null_index; /* Index of the null-accepting partition; -1
- * for range partitioned tables */
+ * for hash and range partitioned tables */
} PartitionBoundInfoData;
/*
@@ -97,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
{
@@ -113,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,
@@ -128,12 +146,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +169,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
bool found_null = false;
@@ -237,7 +266,33 @@ 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 = 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;
@@ -462,6 +517,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->has_null = found_null;
@@ -615,53 +706,111 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->strategy == PARTITION_STRATEGY_HASH)
{
- int j;
+ int greatest_modulus;
- for (j = 0; j < key->partnatts; j++)
+ /*
+ * Compare greatest modulus of hash partition bound which
+ * is the last element of datums array.
+ */
+ if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
+ return false;
+
+ /* Compare number of partitions */
+ if (b1->ndatums != b2->ndatums)
+ return false;
+
+ /*
+ * If two hash partitioned tables have different greatest moduli or
+ * same moduli with different number of partitions, their partition
+ * schemes don't match. If they have same greatest moduli, and number
+ * of partitions, they all have same hash partition bound i.e. modulus
+ * and remainder, and the partitions are ordered by modulus, then by
+ * remainders, thus indexes array will be an identity i.e. index[i] = i.
+ * If the partition corresponding to a given remainder exists, it will
+ * have same index entry for both partitioned tables or if it's missing
+ * it will be -1. Thus if indexes array matches, corresponding datums
+ * array matches. If there are multiple remainders corresponding to a
+ * given partition, their partitions are ordered by the hash partition
+ * bound, thus if indexes array matches, both of the tables have same
+ * indexes arrays, in both the tables remainders corresponding to
+ * multiple partitions all have same hash partition bound and thus same
+ * modulus. Thus again if the indexes are same, datums are same.
+ */
+ 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
+ {
+ /*
+ * 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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -683,6 +832,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -905,6 +1137,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1298,6 +1535,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2307,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
* A null partition key is only acceptable if null-accepting
@@ -2051,6 +2404,34 @@ get_partition_for_tuple(PartitionDispatch *pd,
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2227,6 +2608,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],
@@ -2310,3 +2700,99 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 cdcb949..a71c2f3 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -823,7 +823,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13165,6 +13165,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13197,6 +13199,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13209,6 +13212,19 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
assign_expr_collations(pstate, pelem->expr);
}
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not for equality, is
+ * irrelevant for hash partitioning.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot use collation for hash partition key column \"%s\"",
+ pelem->name),
+ parser_errposition(pstate, pelem->location)));
+
+
newspec->partParams = lappend(newspec->partParams, pelem);
}
@@ -13220,10 +13236,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13366,25 +13385,38 @@ 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 opclass to use. For list and range partitioning we use only
+ * btree operator class, which seems enough for those. For hash partitioning,
+ * we use hash operator class.
*/
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d13a6fc..76baeae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4446,6 +4446,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 c9a8c34..200ed7a 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 3d5b09a..08bca2b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3545,6 +3545,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 f9a227e..ede6306 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2375,6 +2375,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 2822331..15324e5 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @1;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 882955b..628f907 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3282,7 +3282,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be greater than 0")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be a non-zero positive integer")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 c9bded0..74ce07d 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -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)
appendStringInfo(&buf, "LIST");
@@ -8651,6 +8655,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ae37302..73fb3bf 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2022,7 +2022,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
* partitions of <foo>.
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 7434696..d7ad3ef 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 d396be3..6017910 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 41df9f0..8d24d8b 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3198,6 +3198,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,
@@ -3276,6 +3277,54 @@ 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
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+ERROR: partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+ERROR: partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ERROR: hash partition modulus must be greater than 0
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus must be greater than remainder
+ALTER TABLE hash_parted2 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
--
@@ -3287,12 +3336,19 @@ 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_parted2 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_parted2 DETACH PARTITION not_a_part;
+ERROR: relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR: relation "hpart_1" is not a partition of relation "hash_parted2"
+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;
@@ -3375,6 +3431,8 @@ 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, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 bbf039c..a316f67 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -310,6 +310,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
ERROR: column "a" appears more than once in partition key
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+ERROR: cannot use collation for hash partition key column "a"
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -340,11 +347,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
@@ -479,6 +481,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: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
-- specified literal can't be cast to the partition column data type
CREATE TABLE bools (
a bool
@@ -512,6 +519,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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: CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES ...
+ ^
+-- 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 (
@@ -519,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 (
@@ -526,6 +559,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -533,6 +568,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -540,6 +577,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES W...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -599,6 +640,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus must be greater than 0
+-- 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -747,6 +805,8 @@ Partition constraint: ((abs(a) IS NOT NULL) AND (abs(b) IS NOT NULL) AND ((abs(a
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 02429a3..b9ac539 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,8 +313,98 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (b, 21).
+insert into hpart3 values ('a', 10);
+ERROR: new row for relation "hpart3" violates partition constraint
+DETAIL: Failing row contains (a, 10).
+-- ok
+insert into hpart3 values ('c', 6);
+-- fail
+insert into hpart2 values (1);
+ERROR: new row for relation "hpart2" violates partition constraint
+DETAIL: Failing row contains (1, null).
+-- ok
+insert into hpart1 values (1);
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+ERROR: no partition of relation "hash_parted" found for row
+DETAIL: Partition key of the failing row contains (a, b) = (c, 5).
+insert into hpart1 values (null);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (null, null).
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+ part | a
+--------+----
+ hpart1 | 4
+ hpart1 | 8
+ hpart1 | 12
+ hpart1 | 16
+ hpart2 | 1
+ hpart2 | 5
+ hpart2 | 9
+ hpart3 | 2
+ hpart3 | 6
+ hpart3 | 10
+ hpart4 | 3
+ hpart4 | 7
+ hpart4 | 11
+(13 rows)
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..352e87a 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,26 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
-- cleanup
drop table range_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 24d1d4d..f6b95ac 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2065,6 +2065,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 (
@@ -2150,6 +2151,57 @@ 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
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE hash_parted (
+ a int NOT NULL,
+ b char(2) COLLATE "C",
+ CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching hash partitions
+CREATE TABLE hash_parted2 (
+ a int,
+ b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 FOR VALUES WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+ LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 FOR VALUES WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 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_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part FOR VALUES WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
--
-- DETACH PARTITION
--
@@ -2161,12 +2213,17 @@ DROP TABLE regular_table;
-- check that the partition being detached exists at all
ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 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_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+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;
@@ -2228,6 +2285,8 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
-- cleanup
DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
-- 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 766f35a..834cba2 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -320,6 +320,13 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
+-- cannot have collation for hash partition key column (although grammar allows).
+-- Since hash opclasses provide only equality, not ordering, so that collation
+-- is irrelevant here.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -350,11 +357,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
@@ -451,6 +453,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 (
@@ -478,6 +482,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -485,6 +505,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
@@ -492,6 +513,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -499,6 +521,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -507,6 +530,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -558,6 +582,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -627,6 +666,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 db8967b..29a9704 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,8 +185,78 @@ 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 table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart3 values ('b', 21);
+insert into hpart3 values ('a', 10);
+-- ok
+insert into hpart3 values ('c', 6);
+
+-- fail
+insert into hpart2 values (1);
+
+-- ok
+insert into hpart1 values (1);
+
+-- fail due to no partition found.
+insert into hash_parted values ('c', 5);
+insert into hpart1 values (null);
+
+-- to fix above error add new partition with (modulus 8, remainder 0) bound
+create table hpart4 partition of hash_parted for values with (modulus 8, remainder 0);
+
+-- ok
+insert into hash_parted values ('c', 5);
+insert into hpart4 values (null);
+
-- cleanup
drop table range_parted, list_parted;
+drop table hash_parted;
+
+-- Operator class test
+create or replace function opcl_test(a int4) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 opcl_test(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 values(11);
+
+-- view data
+select tableoid::regclass as part, a from hash_parted order by part;
+
+-- cleanup
+drop table hash_parted;
+drop operator class custom_opclass using hash;
+drop function opcl_test(int4);
-- 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..e875192 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
-- ok
update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+ a text,
+ b int
+) partition by hash (a, b);
+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 ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+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, hpart1, hpart2, hpart3, hpart4;
--
2.6.2
0001-Cleanup_v2.patchapplication/octet-stream; name=0001-Cleanup_v2.patchDownload
From 8ef84a31e61271c007852ec100be63740b94a5b9 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Sat, 13 May 2017 18:39:53 +0530
Subject: [PATCH 1/2] Cleanup_v2
Code refactoring required for hash partitioning patch v4
---
src/backend/catalog/partition.c | 87 ++++++++++++++++++++++-------------------
1 file changed, 47 insertions(+), 40 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 885c533..5566839 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -252,8 +252,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -334,8 +333,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1924,10 +1922,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,40 +1965,51 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
- {
- /* Disallow nulls in the range partition key of the tuple */
- for (i = 0; i < key->partnatts; i++)
- if (isnull[i])
- ereport(ERROR,
- (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
- errmsg("range partition key of row contains null")));
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partdesc->boundinfo->has_null)
- 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:
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
+ {
+ if (partdesc->boundinfo->has_null)
+ cur_index = partdesc->boundinfo->null_index;
+ }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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 i;
+ int cur_offset;
- case PARTITION_STRATEGY_RANGE:
+ /* Disallow nulls in the range partition key of the tuple */
+ for (i = 0; i < key->partnatts; i++)
+ if (isnull[i])
+ ereport(ERROR,
+ (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
+ errmsg("range partition key of row contains null")));
+
+ /* 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
@@ -2010,12 +2017,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
On Wed, May 17, 2017 at 2:07 PM, amul sul <sulamul@gmail.com> wrote:
In partition_bounds_equal(), please add comments explaining why is it safe to
check just the indexes? May be we should add code under assertion to make sure
that the datums are equal as well.Added assert in the attached version.
The comment could be something
like, "If two partitioned tables have different greatest moduli, their
partition schemes don't match. If they have same greatest moduli, and
all remainders have different indexes, they all have same modulus
specified and the partitions are ordered by remainders, thus indexes
array will be an identity i.e. index[i] = i. If the partition
corresponding to a given remainder exists, it will have same index
entry for both partitioned tables or if it's missing it will be -1.
Thus if indexes array matches, corresponding datums array matches. If
there are multiple remainders corresponding to a given partition,
their partitions are ordered by the lowest of the remainders, thus if
indexes array matches, both of the tables have same indexes arrays, in
both the tables remainders corresponding to multiple partitions all
have same indexes and thus same modulus. Thus again if the indexes are
same, datums are same.".Thanks, added with minor modification.
I have reworded this slightly better. See the attached patch as diff of 0002.
In the same function
if (key->strategy == PARTITION_STRATEGY_HASH)
{
int greatest_modulus;/*
* Compare greatest modulus of hash partition bound which
* is the last element of datums array.
*/
if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
return false;/* Compare indexes */
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;
}
if we return true from where this block ends, we will save one indenation level
for rest of the code and also FWIW extra diffs in this patch because of this
indentation change.I still do believe having this code in the IF - ELSE block will be
better for longterm, rather having code clutter to avoid diff that
unpleasant for now.
Ok, I will leave it to the committer to judge.
Comments on the tests
+#ifdef USE_ASSERT_CHECKING
+ {
+ /*
+ * 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
Why do we need extra {} here?
Comments on testcases
+CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH
(modulus 8, remainder 0);
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH
(modulus 4, remainder 0);
Probably you should also test the other-way round case i.e. create modulus 4,
remainder 0 partition and then try to add partitions with modulus 8, remainder
4 and modulus 8, remainder 0. That should fail.
Why to create two tables hash_parted and hash_parted2, you should be able to
test with only a single table.
+INSERT INTO hpart_2 VALUES (3, 'a');
+DELETE FROM hpart_2;
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
This is slightly tricky. On different platforms the row may map to different
partitions depending upon how the values are hashed. So, this test may not be
portable on all the platforms. Probably you should add such testcases with a
custom hash operator class which is identity function as suggested by Robert.
This also applies to the tests in insert.sql and update.sql for partitioned
table without custom opclass.
+-- delete the faulting row and also add a constraint to skip the scan
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a
SET NOT NULL;
The constraint is not same as the implicit constraint added for that partition.
I am not sure whether it's really going to avoid the scan. Did you verify it?
If yes, then how?
+ALTER TABLE hash_parted2 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
We should add this test with at least two partitions in there so that we can
check lower and upper modulus. Also, testing with some interesting
bounds discussed earlier
in this mail e.g. adding modulus 15 when 5, 10, 60 exist will be better than
testing with 3, 4 and 8.
+ERROR: cannot use collation for hash partition key column "a"
This seems to indicate that we can not specify collation for hash partition key
column, which isn't true. Column a here can have its collation. What's not
allowed is specifying collation in PARTITION BY clause.
May be reword the error as "cannot use collation for hash partitioning". or
plain "cannot use collation in PARTITION BY clause for hash partitioning".
+ERROR: invalid bound specification for a list partition
+LINE 1: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W...
+ ^
Should the location for this error be that of WITH clause like in case of range
and list partitioned table.
+select tableoid::regclass as part, a from hash_parted order by part;
May be add a % 4 to show clearly that the data really goes to the partitioning
with that remainder.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
0002-extras.patchtext/x-patch; charset=US-ASCII; name=0002-extras.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 5b201d6..15d6170 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -711,32 +711,27 @@ partition_bounds_equal(PartitionKey key,
int greatest_modulus;
/*
- * Compare greatest modulus of hash partition bound which
- * is the last element of datums array.
+ * 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;
- /* Compare number of partitions */
if (b1->ndatums != b2->ndatums)
return false;
/*
- * If two hash partitioned tables have different greatest moduli or
- * same moduli with different number of partitions, their partition
- * schemes don't match. If they have same greatest moduli, and number
- * of partitions, they all have same hash partition bound i.e. modulus
- * and remainder, and the partitions are ordered by modulus, then by
- * remainders, thus indexes array will be an identity i.e. index[i] = i.
- * If the partition corresponding to a given remainder exists, it will
- * have same index entry for both partitioned tables or if it's missing
- * it will be -1. Thus if indexes array matches, corresponding datums
- * array matches. If there are multiple remainders corresponding to a
- * given partition, their partitions are ordered by the hash partition
- * bound, thus if indexes array matches, both of the tables have same
- * indexes arrays, in both the tables remainders corresponding to
- * multiple partitions all have same hash partition bound and thus same
- * modulus. Thus again if the indexes are same, datums are same.
+ * 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++)
@@ -745,9 +740,12 @@ partition_bounds_equal(PartitionKey key,
#ifdef USE_ASSERT_CHECKING
{
+
/*
- * Hash partition bound stores modulus and remainder at
- * b1->datums[i][0] and b1->datums[i][1] position respectively.
+ * 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] &&
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index 834cba2..06ee5f0 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -320,9 +320,8 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
--- cannot have collation for hash partition key column (although grammar allows).
--- Since hash opclasses provide only equality, not ordering, so that collation
--- is irrelevant here.
+-- Since hash opclasses provide only equality and not ordering, collation
+-- is irrelevant for hash partitioning.
CREATE TABLE partitioned (
a text
) PARTITION BY HASH (a collate "C");
On Wed, May 17, 2017 at 1:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Fixed in the attached version; used "hash partition remainder must be
greater than or equal to 0" instead.I would suggest "non-zero positive", since that's what we are using in
the documentation.
Well, that's not very good terminology, because zero is not a positive
number. Existing error messages seem to use phrasing such as "THING
must be a positive integer" when zero is not allowed or "THING must be
a non-negative integer" when zero is allowed. For examples, do git
grep errmsg.*positive or git grep errmsg.*negative.
In partition_bounds_equal(), please add comments explaining why is it safe to
check just the indexes? May be we should add code under assertion to make sure
that the datums are equal as well. The comment could be something
like, "If two partitioned tables have different greatest moduli, their
partition schemes don't match. If they have same greatest moduli, and
all remainders have different indexes, they all have same modulus
specified and the partitions are ordered by remainders, thus indexes
array will be an identity i.e. index[i] = i. If the partition
corresponding to a given remainder exists, it will have same index
entry for both partitioned tables or if it's missing it will be -1.
Thus if indexes array matches, corresponding datums array matches. If
there are multiple remainders corresponding to a given partition,
their partitions are ordered by the lowest of the remainders, thus if
indexes array matches, both of the tables have same indexes arrays, in
both the tables remainders corresponding to multiple partitions all
have same indexes and thus same modulus. Thus again if the indexes are
same, datums are same.".
That seems quite long.
--
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, May 17, 2017 at 11:51 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, May 17, 2017 at 1:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:Fixed in the attached version; used "hash partition remainder must be
greater than or equal to 0" instead.I would suggest "non-zero positive", since that's what we are using in
the documentation.Well, that's not very good terminology, because zero is not a positive
number. Existing error messages seem to use phrasing such as "THING
must be a positive integer" when zero is not allowed or "THING must be
a non-negative integer" when zero is allowed. For examples, do git
grep errmsg.*positive or git grep errmsg.*negative.
Ok. We need to change all the usages in the documentation and in the
comments to non-negative. The point is to use same phrases
consistently.
In partition_bounds_equal(), please add comments explaining why is it safe to
check just the indexes? May be we should add code under assertion to make sure
that the datums are equal as well. The comment could be something
like, "If two partitioned tables have different greatest moduli, their
partition schemes don't match. If they have same greatest moduli, and
all remainders have different indexes, they all have same modulus
specified and the partitions are ordered by remainders, thus indexes
array will be an identity i.e. index[i] = i. If the partition
corresponding to a given remainder exists, it will have same index
entry for both partitioned tables or if it's missing it will be -1.
Thus if indexes array matches, corresponding datums array matches. If
there are multiple remainders corresponding to a given partition,
their partitions are ordered by the lowest of the remainders, thus if
indexes array matches, both of the tables have same indexes arrays, in
both the tables remainders corresponding to multiple partitions all
have same indexes and thus same modulus. Thus again if the indexes are
same, datums are same.".That seems quite long.
I have shared a patch containing a denser explanation with my last set
of comments.
--
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 Wed, May 17, 2017 at 2:07 PM, amul sul <sulamul@gmail.com> wrote:
I would suggest "non-zero positive", since that's what we are using in
the documentation.Understood, Fixed in the attached version.
Why non-zero positive? We do support zero for the remainder right?
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 2017/05/19 1:09, Dilip Kumar wrote:
On Wed, May 17, 2017 at 2:07 PM, amul sul <sulamul@gmail.com> wrote:
I would suggest "non-zero positive", since that's what we are using in
the documentation.Understood, Fixed in the attached version.
Why non-zero positive? We do support zero for the remainder right?
Using "non-negative integers" (for remainders) was suggested upthread.
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 Wed, May 17, 2017 at 6:54 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
[...]
Comments on the tests +#ifdef USE_ASSERT_CHECKING + { + /* + * 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 Why do we need extra {} here?
Okay, removed in the attached version.
Comments on testcases +CREATE TABLE hpart_1 PARTITION OF hash_parted FOR VALUES WITH (modulus 8, remainder 0); +CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS); +ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 4, remainder 0); Probably you should also test the other-way round case i.e. create modulus 4, remainder 0 partition and then try to add partitions with modulus 8, remainder 4 and modulus 8, remainder 0. That should fail.
Fixed.
Why to create two tables hash_parted and hash_parted2, you should be able to
test with only a single table.
Fixed.
+INSERT INTO hpart_2 VALUES (3, 'a'); +DELETE FROM hpart_2; +INSERT INTO hpart_5_a (a, b) VALUES (6, 'a'); This is slightly tricky. On different platforms the row may map to different partitions depending upon how the values are hashed. So, this test may not be portable on all the platforms. Probably you should add such testcases with a custom hash operator class which is identity function as suggested by Robert. This also applies to the tests in insert.sql and update.sql for partitioned table without custom opclass.
Yes, you are correct. Fixed in the attached version.
+-- delete the faulting row and also add a constraint to skip the scan +ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL; The constraint is not same as the implicit constraint added for that partition. I am not sure whether it's really going to avoid the scan. Did you verify it? If yes, then how?
I haven't tested that, may be I've copied blindly, sorry about that.
I don't think this test is needed again for hash partitioning, so removed.
+ALTER TABLE hash_parted2 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 We should add this test with at least two partitions in there so that we can check lower and upper modulus. Also, testing with some interesting bounds discussed earlier in this mail e.g. adding modulus 15 when 5, 10, 60 exist will be better than testing with 3, 4 and 8.
Similar test do exists in create_table.sql file.
+ERROR: cannot use collation for hash partition key column "a"
This seems to indicate that we can not specify collation for hash partition key
column, which isn't true. Column a here can have its collation. What's not
allowed is specifying collation in PARTITION BY clause.
May be reword the error as "cannot use collation for hash partitioning". or
plain "cannot use collation in PARTITION BY clause for hash partitioning".+ERROR: invalid bound specification for a list partition +LINE 1: CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES W... + ^ Should the location for this error be that of WITH clause like in case of range and list partitioned table.
Fixed.
+select tableoid::regclass as part, a from hash_parted order by part;
May be add a % 4 to show clearly that the data really goes to the partitioning
with that remainder.
Fixed.
Updated patch attached. 0001-patch rebased against latest head.
0002-patch also incorporates code comments and error message changes
as per Robert's & your suggestions. Thanks !
Regards,
Amul
Attachments:
0001-Cleanup_v3.patchapplication/octet-stream; name=0001-Cleanup_v3.patchDownload
From 70eeb8ba3aa080d32cadfb0c51b56904268fbf46 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 19 May 2017 14:18:14 +0530
Subject: [PATCH 1/2] Cleanup_v3
Code refactoring required for hash partitioning patch v10.
---
src/backend/catalog/partition.c | 100 ++++++++++++++++++++++------------------
1 file changed, 54 insertions(+), 46 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 7304f6c..66c1071 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -250,8 +250,7 @@ RelationBuildPartitionDesc(Relation rel)
ListCell *c;
PartitionBoundSpec *spec = lfirst(cell);
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
foreach(c, spec->listdatums)
{
@@ -331,8 +330,7 @@ RelationBuildPartitionDesc(Relation rel)
PartitionRangeBound *lower,
*upper;
- if (spec->strategy != PARTITION_STRATEGY_RANGE)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
lower = make_one_range_bound(key, i, spec->lowerdatums,
true);
@@ -1923,10 +1921,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,62 +1965,74 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
+ switch (key->strategy)
{
- /*
- * 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])
+ case PARTITION_STRATEGY_LIST:
+
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
{
- *failed_at = parent;
- *failed_slot = slot;
- result = -1;
- goto error_exit;
+ if (partition_bound_accepts_nulls(partdesc->boundinfo))
+ cur_index = partdesc->boundinfo->null_index;
}
- }
- }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partition_bound_accepts_nulls(partdesc->boundinfo))
- cur_index = partdesc->boundinfo->null_index;
- else if (!isnull[0])
- {
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
+ /* bsearch in partdesc->boundinfo */
+ cur_offset = partition_bound_bsearch(key,
+ partdesc->boundinfo,
+ values, false, &equal);
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
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 i;
+ int cur_offset;
- case 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;
+ }
+ }
+ /* 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
* found to be less or equal with the tuple. So, the bound
* 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-v10.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v10.patchDownload
From 7cc29988796c4fd4be968b0bbc39e2f624966d01 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 19 May 2017 14:19:46 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v10
---
doc/src/sgml/ddl.sgml | 29 +-
doc/src/sgml/ref/alter_table.sgml | 7 +
doc/src/sgml/ref/create_table.sgml | 71 +++-
src/backend/catalog/partition.c | 570 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 53 ++-
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 | 61 ++-
src/backend/parser/parse_utilcmd.c | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 66 +++-
src/test/regress/expected/create_table.out | 69 +++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 68 +++-
src/test/regress/sql/create_table.sql | 50 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
23 files changed, 1152 insertions(+), 88 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..8159730 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -2902,8 +2916,9 @@ VALUES ('Albany', NULL, NULL, 'NY');
<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.
+ partitioning methods include hash, range and list, where each partition is
+ assigned a modulus and remainder of keys, a range of keys and a list of
+ keys, respectively.
</para>
<para>
@@ -3327,11 +3342,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 0478e40..d8142bb 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,17 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. List and range partitioning uses only btree operator class.
+ Hash partitioning uses only hash operator class. If no operator class is
+ specified when creating a partitioned table, the default operator class
+ for the datatype will be used. If there is none, an error will be
+ reported.
+ </para>
+
+ <para>
+ Since hash partitiong operator class, provide only equality, not ordering,
+ collation is not relevant in hash partition key column. An error will be
+ reported if collation is specified.
</para>
<para>
@@ -1586,6 +1618,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
@@ -1636,6 +1678,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 66c1071..a5ed275 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,18 +88,16 @@ typedef enum RangeDatumContent
typedef struct PartitionBoundInfoData
{
- char strategy; /* list or range bounds? */
+ char strategy; /* hash, list or range bounds? */
int ndatums; /* Length of the datums following array */
- Datum **datums; /* Array of datum-tuples with key->partnatts
- * datums each */
+ Datum **datums;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); 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 */
+ * if there isn't one, or is a hash or range
+ * partitioned table */
} PartitionBoundInfoData;
#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
@@ -97,6 +107,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
{
@@ -113,6 +131,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,
@@ -128,12 +147,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -148,6 +170,11 @@ static int partition_bound_bsearch(PartitionKey key,
PartitionBoundInfo boundinfo,
void *probe, bool probe_is_bound, bool *is_equal);
+static uint32 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
* Form rel's partition descriptor
@@ -171,6 +198,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -236,7 +266,33 @@ 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 = 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;
@@ -459,6 +515,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));
@@ -606,53 +698,106 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->partnatts; j++)
+ if (b1->ndatums != b2->ndatums)
+ 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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -674,6 +819,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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_bound = 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 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_bound = (next_modulus % spec->modulus) == 0;
+ }
+ else
+ {
+ prev_modulus = DatumGetInt32(datums[offset][0]);
+ valid_bound = (spec->modulus % prev_modulus) == 0;
+
+ if (valid_bound && (offset + 1) < ndatums)
+ {
+ next_modulus = DatumGetInt32(datums[offset + 1][0]);
+ valid_bound = (next_modulus % spec->modulus) == 0;
+ }
+ }
+
+ if (!valid_bound)
+ 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);
@@ -897,6 +1125,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1290,6 +1523,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1967,6 +2303,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2020,7 +2369,8 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key,
partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
+
/*
* Offset returned is such that the bound at offset is
* found to be less or equal with the tuple. So, the bound
@@ -2062,6 +2412,34 @@ error_exit:
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2238,6 +2616,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],
@@ -2321,3 +2708,100 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 99c51b8..9fcec72 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -821,7 +821,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13128,6 +13128,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13160,6 +13162,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13172,6 +13175,18 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
assign_expr_collations(pstate, pelem->expr);
}
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not for equality, is
+ * irrelevant for hash partitioning.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot use collation for hash partitioning"),
+ parser_errposition(pstate, pelem->location)));
+
+
newspec->partParams = lappend(newspec->partParams, pelem);
}
@@ -13183,10 +13198,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13329,25 +13347,38 @@ 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 opclass to use. For list and range partitioning we use
+ * only btree operator class, which seems enough for those. For hash
+ * partitioning, we use hash operator class.
*/
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_method, am_method)));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 7811ad5..70e8dd4 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4444,6 +4444,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 c9a8c34..200ed7a 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 4949d58..369b3b0 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3543,6 +3543,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 e24f5d6..7ab8324 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,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 2822331..fc3ce4b 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,46 @@ alter_identity_column_option:
;
ForValues:
+ /* a HASH partition*/
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+ {
+ ListCell *lc;
+ PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+ n->strategy = PARTITION_STRATEGY_HASH;
+ n->modulus = n->remainder = -1;
+
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ 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("hash partition modulus must be specified")));
+ if (n->remainder == -1)
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("hash partition remainder must be specified")));
+
+ n->location = @3;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2717,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 beb0995..bce1ff9 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3294,7 +3294,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus must be a positive integer")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition remainder must be a non-negative integer")));
+
+ if (spec->remainder >= spec->modulus)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("hash partition modulus 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 9234bc2..6914528 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1652,6 +1652,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)
appendStringInfo(&buf, "LIST");
@@ -8649,6 +8653,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 2abd087..d563746 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2046,7 +2046,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/pg_proc.h b/src/include/catalog/pg_proc.h
index 460cdb9..de09809 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 4b8727e..6573114 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index c88fd76..8d7f659 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3224,6 +3224,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,
@@ -3302,6 +3303,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, row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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: hash partition modulus must be a positive integer
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES WITH (modulus 8, remainder 8);
+ERROR: hash partition modulus 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
--
@@ -3313,12 +3367,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;
@@ -3338,9 +3397,9 @@ SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::r
DROP TABLE part_3_4;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
-ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
+ALTER TABLE ONLY list_parted ADD COLUMN c int;
ERROR: column must be added to child tables too
-ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY list_parted DROP COLUMN b;
ERROR: cannot drop column from only the partitioned table when partitions exist
HINT: Do not specify the ONLY keyword.
-- cannot add a column to partition or drop an inherited one
@@ -3401,6 +3460,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(int4);
-- 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 39edf04..31749eb 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -310,6 +310,12 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
ERROR: column "a" appears more than once in partition key
+-- Since hash opclasses provide only equality and not ordering, collation
+-- is irrelevant for hash partitioning.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+ERROR: cannot use collation for hash partitioning
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -340,11 +346,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 +468,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
@@ -500,6 +506,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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 (
@@ -507,6 +537,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 (
@@ -514,6 +546,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -521,6 +555,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -528,6 +564,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: ...BLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modu...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -587,6 +627,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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: hash partition modulus 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: hash partition modulus must be greater than remainder
-- check schema propagation from parent
CREATE TABLE parted (
a text,
@@ -735,6 +792,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 8b0752a..97cbbca 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index c0e2972..9edbdc9 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2075,6 +2075,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 (
@@ -2160,6 +2161,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, row may map to different partitions, which result in
+-- regression failure. To avoid this, let's create non-default hash function
+-- that just returns the input value unchanged.
+CREATE OR REPLACE FUNCTION dummy_hashint4(a int4) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2171,12 +2228,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;
@@ -2187,8 +2248,8 @@ DROP TABLE part_3_4;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
-ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
-ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY list_parted ADD COLUMN c int;
+ALTER TABLE ONLY list_parted DROP COLUMN b;
-- cannot add a column to partition or drop an inherited one
ALTER TABLE part_2 ADD COLUMN c text;
@@ -2238,6 +2299,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(int4);
-- 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 5a27743..c4d60ec 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -320,6 +320,12 @@ CREATE TABLE partitioned (
a int
) PARTITION BY RANGE (a, a);
+-- Since hash opclasses provide only equality and not ordering, collation
+-- is irrelevant for hash partitioning.
+CREATE TABLE partitioned (
+ a text
+) PARTITION BY HASH (a collate "C");
+
-- prevent using prohibited expressions in the key
CREATE FUNCTION retset (a int) RETURNS SETOF int AS $$ SELECT 1; $$ LANGUAGE SQL IMMUTABLE;
CREATE TABLE partitioned (
@@ -350,11 +356,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 +447,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 (
@@ -473,6 +476,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -480,6 +499,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
@@ -487,6 +507,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -494,6 +515,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -502,6 +524,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -553,6 +576,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -622,6 +660,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 db8967b..7a147e4 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2
On Fri, May 19, 2017 at 5:32 AM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached. 0001-patch rebased against latest head.
0002-patch also incorporates code comments and error message changes
as per Robert's & your suggestions. Thanks !
- if (spec->strategy != PARTITION_STRATEGY_LIST)
- elog(ERROR, "invalid strategy in partition bound spec");
+ Assert(spec->strategy == PARTITION_STRATEGY_LIST);
Let's just drop these hunks. I realize this is a response to a review
comment I made, but I take it back. If the existing code is already
doing it this way, there's no real need to revise it. The patch
doesn't even make it consistent anyway, since elsewhere you elog() for
a similar case. Perhaps elog() is best anyway.
- partitioning methods include range and list, where each partition is
- assigned a range of keys and a list of keys, respectively.
+ partitioning methods include hash, range and list, where each partition is
+ assigned a modulus and remainder of keys, a range of keys and a list of
+ keys, respectively.
I think this sentence has become too long and unwieldy, and is more
unclear than helpful. I'd just write "The currently supported
partitioning methods are list, range, and hash." The use of the word
include is actually wrong here, because it implies that there are more
not mentioned here, which is false.
- expression. If no btree operator class is specified when creating a
- partitioned table, the default btree operator class for the datatype will
- be used. If there is none, an error will be reported.
+ expression. List and range partitioning uses only btree operator class.
+ Hash partitioning uses only hash operator class. If no operator class is
+ specified when creating a partitioned table, the default operator class
+ for the datatype will be used. If there is none, an error will be
+ reported.
+ </para>
I suggest: 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>
+ Since hash partitiong operator class, provide only equality,
not ordering,
+ collation is not relevant in hash partition key column. An error will be
+ reported if collation is specified.
partitiong -> partitioning. Also, remove the comma after "operator
class" and change "not relevant in hash partition key column" to "not
relevant for hash partitioning". Also change "if collation is
specified" to "if a collation is specified".
+ 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>
Move this down so it's just above the example of creating 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.
Second line is very short; reflow as one paragraph.
* 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.
+ * 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.
Insert line break before the new text as a paragraph break.
+ char strategy; /* hash, list or range bounds? */
Might be clearer to just write /* hash, list, or range? */ or /*
bounds for hash, list, or range? */
+static uint32 compute_hash_value(PartitionKey key, Datum *values,
bool *isnull);
+
I think there should be a blank line after this but not before it.
I don't really see why hash partitioning needs to touch
partition_bounds_equal() at all. Why can't the existing logic work
for hash partitioning without change?
+ valid_bound = true;
valid_modulus, maybe?
- 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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_method),
+ errhint("You must specify a %s operator
class or define a default %s operator class for the data type.",
+ am_method, am_method)));
Let's use this existing wording from typecmds.c:
errmsg("data type %s has no default operator
class for access method \"%s\"",
and for the hint, maybe: You must specify an operator class or define
a default operator class for the data type. Leave out the %s, in
other words.
+ /*
+ * Hash operator classes provide only equality, not ordering.
+ * Collation, which is relevant for ordering and not for equality, is
+ * irrelevant for hash partitioning.
+ */
+ if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot use collation for hash partitioning"),
+ parser_errposition(pstate, pelem->location)));
This error message is not very informative, and it requires
propagating information about the partitioning type into parts of the
code that otherwise don't require it. I was waffling before on
whether to ERROR here; I think now I'm in favor of ignoring the
problem. The collation won't do any harm; it just won't affect the
behavior.
+ * Identify opclass to use. For list and range partitioning we use
+ * only btree operator class, which seems enough for those. For hash
+ * partitioning, we use hash operator class.
Strange wording. Suggest: Identify the appropriate operator class.
For list and range partitioning, we use a btree operator class; hash
partitioning uses a hash operator class.
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is
not finalised*/
Remove the comment.
+ foreach (lc, $5)
+ {
+ DefElem *opt = (DefElem *) lfirst(lc);
+
+ if (strcmp(opt->defname, "modulus") == 0)
+ n->modulus = defGetInt32(opt);
+ else if (strcmp(opt->defname, "remainder") == 0)
+ n->remainder = defGetInt32(opt);
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_SYNTAX_ERROR),
+ errmsg("unrecognized hash
partition bound specification \"%s\"",
+ opt->defname),
+ parser_errposition(opt->location)));
+ }
This logic doesn't complain if the same option is specified more than
once. I suggest adding a check for that, and also pushing this logic
out into a helper function that gets called here instead of including
it inline.
+ errmsg("hash partition modulus must be a positive
integer")));
modulus for hash partition
+ errmsg("hash partition remainder must be a
non-negative integer")));
remainder for hash partition
+ errmsg("hash partition modulus must be greater than remainder")));
modulus for hash partition must be greater than remainder
+-- values are hashed, row may map to different partitions, which result in
the row
+-- regression failure. To avoid this, let's create non-default hash function
create a non-default
--
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, May 19, 2017 at 10:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
+ * 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.Second line is very short; reflow as one paragraph.
* 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. + * 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.Insert line break before the new text as a paragraph break.
The prologue is arranged as one paragraph (with a new line) per
member. Within each paragraph explanation for each partitioning
strategy starts on its own line. One paragraph per member is more
readable than separate paragraphs for each member and strategy.
I don't really see why hash partitioning needs to touch
partition_bounds_equal() at all. Why can't the existing logic work
for hash partitioning without change?
Right now, it compares partnatts datums values for list and range. For
hash it requires to compare 2 datums remainder and modulus. So, the
difference?
Further, I suggested that we use the fact that equality of indexes
array implies equality of bounds for hash partitioning.
--
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, May 19, 2017 at 10:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, May 19, 2017 at 5:32 AM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached. 0001-patch rebased against latest head.
0002-patch also incorporates code comments and error message changes
as per Robert's & your suggestions. Thanks !- if (spec->strategy != PARTITION_STRATEGY_LIST) - elog(ERROR, "invalid strategy in partition bound spec"); + Assert(spec->strategy == PARTITION_STRATEGY_LIST);Let's just drop these hunks. I realize this is a response to a review
comment I made, but I take it back. If the existing code is already
doing it this way, there's no real need to revise it. The patch
doesn't even make it consistent anyway, since elsewhere you elog() for
a similar case. Perhaps elog() is best anyway.
Done.
- partitioning methods include range and list, where each partition is - assigned a range of keys and a list of keys, respectively. + partitioning methods include hash, range and list, where each partition is + assigned a modulus and remainder of keys, a range of keys and a list of + keys, respectively.I think this sentence has become too long and unwieldy, and is more
unclear than helpful. I'd just write "The currently supported
partitioning methods are list, range, and hash." The use of the word
include is actually wrong here, because it implies that there are more
not mentioned here, which is false.
Done.
- expression. If no btree operator class is specified when creating a - partitioned table, the default btree operator class for the datatype will - be used. If there is none, an error will be reported. + expression. List and range partitioning uses only btree operator class. + Hash partitioning uses only hash operator class. If no operator class is + specified when creating a partitioned table, the default operator class + for the datatype will be used. If there is none, an error will be + reported. + </para>I suggest: 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.
Done.
+ <para> + Since hash partitiong operator class, provide only equality, not ordering, + collation is not relevant in hash partition key column. An error will be + reported if collation is specified.partitiong -> partitioning. Also, remove the comma after "operator
class" and change "not relevant in hash partition key column" to "not
relevant for hash partitioning". Also change "if collation is
specified" to "if a collation is specified".
Done.
+ 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>Move this down so it's just above the example of creating partitions.
Done.
+ * 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.Second line is very short; reflow as one paragraph.
Done
* 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. + * 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.Insert line break before the new text as a paragraph break.
Will wait for more inputs on Ashutosh's explanation upthread.
+ char strategy; /* hash, list or range bounds? */
Might be clearer to just write /* hash, list, or range? */ or /*
bounds for hash, list, or range? */
Done, used "hash, list, or range?"
+static uint32 compute_hash_value(PartitionKey key, Datum *values, bool *isnull); +I think there should be a blank line after this but not before it.
Done.
I don't really see why hash partitioning needs to touch
partition_bounds_equal() at all. Why can't the existing logic work
for hash partitioning without change?
Unlike list and range partition, ndatums does not represents size of
the indexes array, also dimension of datums array in not the same as
a key->partnatts.
+ valid_bound = true;
valid_modulus, maybe?
Sure, added.
- 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."))); + errmsg("data type %s has no default %s operator class", + format_type_be(atttype), am_method), + errhint("You must specify a %s operator class or define a default %s operator class for the data type.", + am_method, am_method)));Let's use this existing wording from typecmds.c:
errmsg("data type %s has no default operator
class for access method \"%s\"",and for the hint, maybe: You must specify an operator class or define
a default operator class for the data type. Leave out the %s, in
other words.
Done.
+ /* + * Hash operator classes provide only equality, not ordering. + * Collation, which is relevant for ordering and not for equality, is + * irrelevant for hash partitioning. + */ + if (*strategy == PARTITION_STRATEGY_HASH && pelem->collation != NIL) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot use collation for hash partitioning"), + parser_errposition(pstate, pelem->location)));This error message is not very informative, and it requires
propagating information about the partitioning type into parts of the
code that otherwise don't require it. I was waffling before on
whether to ERROR here; I think now I'm in favor of ignoring the
problem. The collation won't do any harm; it just won't affect the
behavior.
Removed.
+ * Identify opclass to use. For list and range partitioning we use + * only btree operator class, which seems enough for those. For hash + * partitioning, we use hash operator class.Strange wording. Suggest: Identify the appropriate operator class.
For list and range partitioning, we use a btree operator class; hash
partitioning uses a hash operator class.
Done
+ FOR VALUES WITH '(' hash_partbound ')' /*TODO: syntax is
not finalised*/Remove the comment.
Done.
+ foreach (lc, $5) + { + DefElem *opt = (DefElem *) lfirst(lc); + + if (strcmp(opt->defname, "modulus") == 0) + n->modulus = defGetInt32(opt); + else if (strcmp(opt->defname, "remainder") == 0) + n->remainder = defGetInt32(opt); + else + ereport(ERROR, + (errcode(ERRCODE_SYNTAX_ERROR), + errmsg("unrecognized hash partition bound specification \"%s\"", + opt->defname), + parser_errposition(opt->location))); + }This logic doesn't complain if the same option is specified more than
once. I suggest adding a check for that, and also pushing this logic
out into a helper function that gets called here instead of including
it inline.
Added duplicate error.
About separate helper function, can't we have as it is, because, imo,
we might not going to use that elsewhere?
+ errmsg("hash partition modulus must be a positive
integer")));modulus for hash partition
+ errmsg("hash partition remainder must be a
non-negative integer")));remainder for hash partition
+ errmsg("hash partition modulus must be greater than remainder")));
modulus for hash partition must be greater than remainder
Done. Similar changes in gram.y as well.
+-- values are hashed, row may map to different partitions, which result in
the row
+-- regression failure. To avoid this, let's create non-default hash function
create a non-default
Done.
Updated patch attached. Thanks a lot for review.
Regards,
Amul
Attachments:
0001-Cleanup_v4.patchapplication/octet-stream; name=0001-Cleanup_v4.patchDownload
From 8c658358c22ed51dea35b21495cfddb2a0ee2459 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 22 May 2017 10:16:00 +0530
Subject: [PATCH 1/2] Cleanup_v4
Code refactoring required for hash partitioning patch v11.
Changes :
Removed hunk as suggested by Robert in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
---
src/backend/catalog/partition.c | 94 +++++++++++++++++++++++------------------
1 file changed, 52 insertions(+), 42 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 7304f6c..763a24b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1923,10 +1923,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,62 +1967,74 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
+ switch (key->strategy)
{
- /*
- * 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])
+ case PARTITION_STRATEGY_LIST:
+
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
{
- *failed_at = parent;
- *failed_slot = slot;
- result = -1;
- goto error_exit;
+ if (partition_bound_accepts_nulls(partdesc->boundinfo))
+ cur_index = partdesc->boundinfo->null_index;
}
- }
- }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partition_bound_accepts_nulls(partdesc->boundinfo))
- cur_index = partdesc->boundinfo->null_index;
- else if (!isnull[0])
- {
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
+ /* bsearch in partdesc->boundinfo */
+ cur_offset = partition_bound_bsearch(key,
+ partdesc->boundinfo,
+ values, false, &equal);
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
if (cur_offset >= 0 && equal)
cur_index = partdesc->boundinfo->indexes[cur_offset];
else
cur_index = -1;
- break;
+ }
+ break;
- case PARTITION_STRATEGY_RANGE:
+ case PARTITION_STRATEGY_RANGE:
+ {
+ bool equal = false;
+ int i;
+ int cur_offset;
/*
+ * 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;
+ }
+ }
+
+ /* 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
* found to be less or equal with the tuple. So, the bound
* 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-v11.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v11.patchDownload
From 36aa78789cb16ead7fcd83bbdfff4c0ffe283ba7 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 19 May 2017 14:19:46 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v11
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 | 70 +++-
src/backend/catalog/partition.c | 569 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 40 +-
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 | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 66 +++-
src/test/regress/expected/create_table.out | 71 +++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 68 +++-
src/test/regress/sql/create_table.sql | 44 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
23 files changed, 1142 insertions(+), 93 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..5bc1a1b 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -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 list, range, and hash.
</para>
<para>
@@ -3327,11 +3340,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 0478e40..9513e88 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,16 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree 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 partitioning operator class provide only equality, not
+ ordering, collation is not relevant for hash partitioning. An error
+ will be reported if a collation is specified.
</para>
<para>
@@ -1636,6 +1667,29 @@ 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 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 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 763a24b..792dd6d 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,18 +88,16 @@ typedef enum RangeDatumContent
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;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); 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 */
+ * if there isn't one, or is a hash or range
+ * partitioned table */
} PartitionBoundInfoData;
#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
@@ -97,6 +107,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
{
@@ -113,6 +131,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,
@@ -128,12 +147,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, RangeDatumContent *content1, 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 uint32 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
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -236,7 +265,33 @@ 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 = 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;
@@ -461,6 +516,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));
@@ -608,53 +699,106 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->partnatts; j++)
+ if (b1->ndatums != b2->ndatums)
+ 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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -676,6 +820,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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 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);
@@ -899,6 +1126,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1292,6 +1524,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1969,6 +2304,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2022,7 +2370,8 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key,
partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
+
/*
* Offset returned is such that the bound at offset is
* found to be less or equal with the tuple. So, the bound
@@ -2064,6 +2413,34 @@ error_exit:
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2240,6 +2617,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],
@@ -2323,3 +2709,100 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 fb961e4..ae7772d 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -822,7 +822,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13129,6 +13129,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13161,6 +13163,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13184,10 +13187,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13330,25 +13336,37 @@ 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_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default operator class for access method \"%s\"",
+ format_type_be(atttype), am_method),
+ errhint("You must specify an operator class or define a default operator class for the data type")));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 7811ad5..70e8dd4 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4444,6 +4444,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 c9a8c34..200ed7a 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 4949d58..369b3b0 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3543,6 +3543,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 e24f5d6..7ab8324 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,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 2822331..d2dde4f 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,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 = (DefElem *) lfirst(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;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2731,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 beb0995..7f79a9f 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3294,7 +3294,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("modulus for hash partition must be a positive integer")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("remainder for hash partition must be a non-negative integer")));
+
+ 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 9234bc2..6914528 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1652,6 +1652,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)
appendStringInfo(&buf, "LIST");
@@ -8649,6 +8653,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 2abd087..d563746 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2046,7 +2046,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/pg_proc.h b/src/include/catalog/pg_proc.h
index 460cdb9..de09809 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 4b8727e..6573114 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index c88fd76..efd8040 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3224,6 +3224,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,
@@ -3302,6 +3303,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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
--
@@ -3313,12 +3367,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;
@@ -3338,9 +3397,9 @@ SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::r
DROP TABLE part_3_4;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
-ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
+ALTER TABLE ONLY list_parted ADD COLUMN c int;
ERROR: column must be added to child tables too
-ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY list_parted DROP COLUMN b;
ERROR: cannot drop column from only the partitioned table when partitions exist
HINT: Do not specify the ONLY keyword.
-- cannot add a column to partition or drop an inherited one
@@ -3401,6 +3460,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(int4);
-- 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 39edf04..b75d650 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
@@ -371,8 +366,8 @@ ERROR: partition key expressions cannot contain whole-row references
CREATE TABLE partitioned (
a point
) PARTITION BY LIST (a);
-ERROR: data type point has no default btree operator class
-HINT: You must specify a btree operator class or define a default btree operator class for the data type.
+ERROR: data type point has no default operator class for access method "btree"
+HINT: You must specify an operator class or define a default operator class for the data type
CREATE TABLE partitioned (
a point
) PARTITION BY LIST (a point_ops);
@@ -380,8 +375,8 @@ ERROR: operator class "point_ops" does not exist for access method "btree"
CREATE TABLE partitioned (
a point
) PARTITION BY RANGE (a);
-ERROR: data type point has no default btree operator class
-HINT: You must specify a btree operator class or define a default btree operator class for the data type.
+ERROR: data type point has no default operator class for access method "btree"
+HINT: You must specify an operator class or define a default operator class for the data type
CREATE TABLE partitioned (
a point
) PARTITION BY RANGE (a point_ops);
@@ -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
@@ -500,6 +500,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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 (
@@ -507,6 +531,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 (
@@ -514,6 +540,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -521,6 +549,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -528,6 +558,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: ...BLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modu...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -587,6 +621,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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,
@@ -735,6 +786,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 8b0752a..97cbbca 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index c0e2972..3d43494 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2075,6 +2075,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 (
@@ -2160,6 +2161,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2171,12 +2228,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;
@@ -2187,8 +2248,8 @@ DROP TABLE part_3_4;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
-ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
-ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY list_parted ADD COLUMN c int;
+ALTER TABLE ONLY list_parted DROP COLUMN b;
-- cannot add a column to partition or drop an inherited one
ALTER TABLE part_2 ADD COLUMN c text;
@@ -2238,6 +2299,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(int4);
-- 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 5a27743..eb909b4 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 (
@@ -473,6 +470,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -480,6 +493,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
@@ -487,6 +501,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -494,6 +509,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -502,6 +518,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -553,6 +570,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -622,6 +654,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 db8967b..7a147e4 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2
On Mon, May 22, 2017 at 2:23 PM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached. Thanks a lot for review.
Minor fix in the document, PFA.
Regards,
Amul
Attachments:
0002-hash-partitioning_another_design-v12.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v12.patchDownload
From 10caae1db2a05fde0ea058917fe067045c05f518 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Fri, 19 May 2017 14:19:46 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v12
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 | 70 +++-
src/backend/catalog/partition.c | 569 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 40 +-
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 | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
src/include/catalog/pg_proc.h | 4 +
src/include/nodes/parsenodes.h | 8 +-
src/test/regress/expected/alter_table.out | 66 +++-
src/test/regress/expected/create_table.out | 71 +++-
src/test/regress/expected/insert.out | 46 +++
src/test/regress/expected/update.out | 29 ++
src/test/regress/sql/alter_table.sql | 68 +++-
src/test/regress/sql/create_table.sql | 44 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
23 files changed, 1142 insertions(+), 93 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..5bc1a1b 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -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 list, range, and hash.
</para>
<para>
@@ -3327,11 +3340,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 0478e40..0fff4dd 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,16 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree 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 partitioning 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>
@@ -1636,6 +1667,29 @@ 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 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 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 763a24b..792dd6d 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,18 +88,16 @@ typedef enum RangeDatumContent
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;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); 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 */
+ * if there isn't one, or is a hash or range
+ * partitioned table */
} PartitionBoundInfoData;
#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
@@ -97,6 +107,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
{
@@ -113,6 +131,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,
@@ -128,12 +147,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, RangeDatumContent *content1, 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 uint32 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
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -236,7 +265,33 @@ 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 = 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;
@@ -461,6 +516,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));
@@ -608,53 +699,106 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->partnatts; j++)
+ if (b1->ndatums != b2->ndatums)
+ 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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -676,6 +820,89 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
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 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);
@@ -899,6 +1126,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
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);
@@ -1292,6 +1524,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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 a list of expressions to use as a list partition's constraint.
@@ -1969,6 +2304,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2022,7 +2370,8 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key,
partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
+
/*
* Offset returned is such that the bound at offset is
* found to be less or equal with the tuple. So, the bound
@@ -2064,6 +2413,34 @@ error_exit:
}
/*
+ * 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);
+}
+
+/*
+ * 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
@@ -2240,6 +2617,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],
@@ -2323,3 +2709,100 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
return lo;
}
+
+/*
+ * This function takes an already computed hash values and combine them
+ * into a single 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 fb961e4..ae7772d 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -822,7 +822,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
&strategy);
ComputePartitionAttrs(rel, stmt->partspec->partParams,
partattrs, &partexprs, partopclass,
- partcollation);
+ partcollation, strategy);
partnatts = list_length(stmt->partspec->partParams);
StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -13129,6 +13129,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (!pg_strcasecmp(partspec->strategy, "range"))
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (!pg_strcasecmp(partspec->strategy, "hash"))
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13161,6 +13163,7 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
errmsg("column \"%s\" appears more than once in partition key",
pelem->name),
parser_errposition(pstate, pelem->location)));
+
}
if (pelem->expr)
@@ -13184,10 +13187,13 @@ 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;
+ char *am_method;
+ Oid am_oid;
attn = 0;
foreach(lc, partParams)
@@ -13330,25 +13336,37 @@ 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_method = "hash";
+ am_oid = HASH_AM_OID;
+ }
+ else
+ {
+ am_method = "btree";
+ 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.")));
+ errmsg("data type %s has no default operator class for access method \"%s\"",
+ format_type_be(atttype), am_method),
+ errhint("You must specify an operator class or define a default operator class for the data type")));
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_method,
+ am_oid);
attn++;
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 7811ad5..70e8dd4 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4444,6 +4444,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 c9a8c34..200ed7a 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 4949d58..369b3b0 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3543,6 +3543,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
WRITE_NODE_TYPE("PARTITIONBOUND");
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 e24f5d6..7ab8324 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,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 2822331..d2dde4f 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 <node> partbound_datum
%type <list> partbound_datum_list
%type <partrange_datum> PartitionRangeDatum
-%type <list> range_datum_list
+%type <list> range_datum_list hash_partbound
+%type <defelt> hash_partbound_elem
/*
* Non-keyword token types. These are hard-wired into the "flex" lexer.
@@ -2652,8 +2653,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 = (DefElem *) lfirst(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;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2678,6 +2731,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 beb0995..7f79a9f 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3294,7 +3294,30 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
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(bound))));
+
+ if (spec->modulus <= 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("modulus for hash partition must be a positive integer")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("remainder for hash partition must be a non-negative integer")));
+
+ 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 9234bc2..6914528 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1652,6 +1652,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)
appendStringInfo(&buf, "LIST");
@@ -8649,6 +8653,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 2abd087..d563746 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2046,7 +2046,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/pg_proc.h b/src/include/catalog/pg_proc.h
index 460cdb9..de09809 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 4b8727e..6573114 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -773,11 +773,13 @@ 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;
+#define PARTITION_STRATEGY_HASH 'h'
#define PARTITION_STRATEGY_LIST 'l'
#define PARTITION_STRATEGY_RANGE 'r'
@@ -790,6 +792,10 @@ typedef struct PartitionBoundSpec
char strategy;
+ /* Hash partition specs */
+ int modulus;
+ int remainder;
+
/* List partition values */
List *listdatums;
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index c88fd76..efd8040 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3224,6 +3224,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,
@@ -3302,6 +3303,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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
--
@@ -3313,12 +3367,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;
@@ -3338,9 +3397,9 @@ SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::r
DROP TABLE part_3_4;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
-ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
+ALTER TABLE ONLY list_parted ADD COLUMN c int;
ERROR: column must be added to child tables too
-ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY list_parted DROP COLUMN b;
ERROR: cannot drop column from only the partitioned table when partitions exist
HINT: Do not specify the ONLY keyword.
-- cannot add a column to partition or drop an inherited one
@@ -3401,6 +3460,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(int4);
-- 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 39edf04..b75d650 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
@@ -371,8 +366,8 @@ ERROR: partition key expressions cannot contain whole-row references
CREATE TABLE partitioned (
a point
) PARTITION BY LIST (a);
-ERROR: data type point has no default btree operator class
-HINT: You must specify a btree operator class or define a default btree operator class for the data type.
+ERROR: data type point has no default operator class for access method "btree"
+HINT: You must specify an operator class or define a default operator class for the data type
CREATE TABLE partitioned (
a point
) PARTITION BY LIST (a point_ops);
@@ -380,8 +375,8 @@ ERROR: operator class "point_ops" does not exist for access method "btree"
CREATE TABLE partitioned (
a point
) PARTITION BY RANGE (a);
-ERROR: data type point has no default btree operator class
-HINT: You must specify a btree operator class or define a default btree operator class for the data type.
+ERROR: data type point has no default operator class for access method "btree"
+HINT: You must specify an operator class or define a default operator class for the data type
CREATE TABLE partitioned (
a point
) PARTITION BY RANGE (a point_ops);
@@ -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
@@ -500,6 +500,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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 (
@@ -507,6 +531,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 (
@@ -514,6 +540,8 @@ CREATE TEMP TABLE temp_parted (
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
+ERROR: cannot create a permanent relation as partition of temporary relation "temp_parted"
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
CREATE TABLE no_oids_parted (
@@ -521,6 +549,8 @@ CREATE TABLE no_oids_parted (
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
ERROR: cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR: cannot create table with OIDs as partition of table without OIDs
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
-- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -528,6 +558,10 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR: invalid bound specification for a range partition
+LINE 1: ...BLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modu...
+ ^
\d+ part_forced_oids
Table "public.part_forced_oids"
Column | Type | Collation | Nullable | Default | Storage | Stats target | Description
@@ -587,6 +621,23 @@ ERROR: partition "fail_part" would overlap partition "part12"
-- more specific ranges
CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
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,
@@ -735,6 +786,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 8b0752a..97cbbca 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index c0e2972..3d43494 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2075,6 +2075,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 (
@@ -2160,6 +2161,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2171,12 +2228,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;
@@ -2187,8 +2248,8 @@ DROP TABLE part_3_4;
-- Check ALTER TABLE commands for partitioned tables and partitions
-- cannot add/drop column to/from *only* the parent
-ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
-ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY list_parted ADD COLUMN c int;
+ALTER TABLE ONLY list_parted DROP COLUMN b;
-- cannot add a column to partition or drop an inherited one
ALTER TABLE part_2 ADD COLUMN c text;
@@ -2238,6 +2299,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(int4);
-- 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 5a27743..eb909b4 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 (
@@ -473,6 +470,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -480,6 +493,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
@@ -487,6 +501,7 @@ CREATE TEMP TABLE temp_parted (
a int
) PARTITION BY LIST (a);
CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES WITH (modulus 2, remainder 1);
DROP TABLE temp_parted;
-- cannot create a table with oids as partition of table without oids
@@ -494,6 +509,7 @@ CREATE TABLE no_oids_parted (
a int
) PARTITION BY RANGE (a) WITHOUT OIDS;
CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITH OIDS;
DROP TABLE no_oids_parted;
-- If the partitioned table has oids, then the partition must have them.
@@ -502,6 +518,7 @@ CREATE TABLE oids_parted (
a int
) PARTITION BY RANGE (a) WITH OIDS;
CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted FOR VALUES WITH (modulus 2, remainder 1) WITHOUT OIDS;
\d+ part_forced_oids
DROP TABLE oids_parted, part_forced_oids;
@@ -553,6 +570,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -622,6 +654,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 db8967b..7a147e4 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2
0001-Cleanup_v4.patchapplication/octet-stream; name=0001-Cleanup_v4.patchDownload
From 8c658358c22ed51dea35b21495cfddb2a0ee2459 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 22 May 2017 10:16:00 +0530
Subject: [PATCH 1/2] Cleanup_v4
Code refactoring required for hash partitioning patch v11.
Changes :
Removed hunk as suggested by Robert in message-id :
CA%2BTgmoabcyyYPe_TRiHyXb%2BAa2rQ%2BzVC9ZHHLASPHmmYbp4sig%40mail.gmail.com
---
src/backend/catalog/partition.c | 94 +++++++++++++++++++++++------------------
1 file changed, 52 insertions(+), 42 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 7304f6c..763a24b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1923,10 +1923,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1969,62 +1967,74 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
+ switch (key->strategy)
{
- /*
- * 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])
+ case PARTITION_STRATEGY_LIST:
+
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
{
- *failed_at = parent;
- *failed_slot = slot;
- result = -1;
- goto error_exit;
+ if (partition_bound_accepts_nulls(partdesc->boundinfo))
+ cur_index = partdesc->boundinfo->null_index;
}
- }
- }
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partition_bound_accepts_nulls(partdesc->boundinfo))
- cur_index = partdesc->boundinfo->null_index;
- else if (!isnull[0])
- {
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
+ /* bsearch in partdesc->boundinfo */
+ cur_offset = partition_bound_bsearch(key,
+ partdesc->boundinfo,
+ values, false, &equal);
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
if (cur_offset >= 0 && equal)
cur_index = partdesc->boundinfo->indexes[cur_offset];
else
cur_index = -1;
- break;
+ }
+ break;
- case PARTITION_STRATEGY_RANGE:
+ case PARTITION_STRATEGY_RANGE:
+ {
+ bool equal = false;
+ int i;
+ int cur_offset;
/*
+ * 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;
+ }
+ }
+
+ /* 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
* found to be less or equal with the tuple. So, the bound
* 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
On Mon, May 22, 2017 at 1:49 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
The prologue is arranged as one paragraph (with a new line) per
member. Within each paragraph explanation for each partitioning
strategy starts on its own line. One paragraph per member is more
readable than separate paragraphs for each member and strategy.
The point is that you can either make it a separate paragraph or you
can make it a single paragraph, but you can't leave it halfway in
between. If it's one paragraph, every line should end at around the
80 character mark, without any short lines. If it's multiple
paragraphs, they should be separated by blank lines. The only line of
a paragraph that can be short is the last 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
On Thu, May 25, 2017 at 9:59 AM, amul sul <sulamul@gmail.com> wrote:
On Mon, May 22, 2017 at 2:23 PM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached. Thanks a lot for review.
Minor fix in the document, PFA.
Patch need rebase
-------
Function header is not consistent with other neighbouring functions
(some function contains function name in the header but others don't)
+/*
+ * Compute the hash value for given not null partition key values.
+ */
------
postgres=# create table t1 partition of t for values with (modulus 2,
remainder 1) partition by range(a);
CREATE TABLE
postgres=# create table t1_1 partition of t1 for values from (8) to (10);
CREATE TABLE
postgres=# insert into t1 values(8);
2017-06-03 18:41:46.067 IST [5433] ERROR: new row for relation "t1_1"
violates partition constraint
2017-06-03 18:41:46.067 IST [5433] DETAIL: Failing row contains (8).
2017-06-03 18:41:46.067 IST [5433] STATEMENT: insert into t1 values(8);
ERROR: new row for relation "t1_1" violates partition constraint
DETAIL: Failing row contains (8).
The value 8 is violating the partition constraint of the t1 and we are
trying to insert to value in t1,
still, the error is coming from the leaf level table t1_1, that may be
fine but from error, it appears that
it's violating the constraint of t1_1 whereas it's actually violating
the constraint of t1.
From Implementation, it appears that based on the key are identifying
the leaf partition and it's only failing during ExecInsert while
checking the partition constraint.
Other than that, patch looks fine to me.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi Dilip,
Thanks for review.
On Sat, Jun 3, 2017 at 6:54 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Thu, May 25, 2017 at 9:59 AM, amul sul <sulamul@gmail.com> wrote:
On Mon, May 22, 2017 at 2:23 PM, amul sul <sulamul@gmail.com> wrote:
Updated patch attached. Thanks a lot for review.
Minor fix in the document, PFA.
Patch need rebase
Done.
------- Function header is not consistent with other neighbouring functions (some function contains function name in the header but others don't) +/* + * Compute the hash value for given not null partition key values. + */
Done.
------
postgres=# create table t1 partition of t for values with (modulus 2,
remainder 1) partition by range(a);
CREATE TABLE
postgres=# create table t1_1 partition of t1 for values from (8) to (10);
CREATE TABLE
postgres=# insert into t1 values(8);
2017-06-03 18:41:46.067 IST [5433] ERROR: new row for relation "t1_1"
violates partition constraint
2017-06-03 18:41:46.067 IST [5433] DETAIL: Failing row contains (8).
2017-06-03 18:41:46.067 IST [5433] STATEMENT: insert into t1 values(8);
ERROR: new row for relation "t1_1" violates partition constraint
DETAIL: Failing row contains (8).The value 8 is violating the partition constraint of the t1 and we are
trying to insert to value in t1,
still, the error is coming from the leaf level table t1_1, that may be
fine but from error, it appears that
it's violating the constraint of t1_1 whereas it's actually violating
the constraint of t1.From Implementation, it appears that based on the key are identifying
the leaf partition and it's only failing during ExecInsert while
checking the partition constraint.
May I ask you, how you sure about 8 is an unfit value for t1 relation?
And what if the value other than 8, for e.g. 7?
Updated patch attached.
Regards,
Amul Sul
Attachments:
0001-Cleanup_v5.patchapplication/octet-stream; name=0001-Cleanup_v5.patchDownload
From 3bf64687db473674e04a105face0a5a588b70aec Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 6 Jun 2017 10:52:39 +0530
Subject: [PATCH 1/2] Cleanup_v5
Code refactoring required for hash partitioning patch v13.
---
src/backend/catalog/partition.c | 93 ++++++++++++++++++++++-------------------
1 file changed, 51 insertions(+), 42 deletions(-)
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 5c5a9e1..8d03292 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1924,10 +1924,8 @@ 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,
- result;
+ int cur_index = -1;
+ int result;
ExprContext *ecxt = GetPerTupleExprContext(estate);
TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
@@ -1970,61 +1968,72 @@ get_partition_for_tuple(PartitionDispatch *pd,
ecxt->ecxt_scantuple = slot;
FormPartitionKeyDatum(parent, slot, estate, values, isnull);
- if (key->strategy == PARTITION_STRATEGY_RANGE)
+ switch (key->strategy)
{
- /*
- * 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])
+ case PARTITION_STRATEGY_LIST:
+
+ /*
+ * A null partition key is only acceptable if null-accepting
+ * list partition exists.
+ */
+ if (isnull[0])
{
- *failed_at = parent;
- *failed_slot = slot;
- result = -1;
- goto error_exit;
+ if (partition_bound_accepts_nulls(partdesc->boundinfo))
+ cur_index = partdesc->boundinfo->null_index;
}
- }
- }
-
- /*
- * A null partition key is only acceptable if null-accepting list
- * partition exists.
- */
- cur_index = -1;
- if (isnull[0] && partition_bound_accepts_nulls(partdesc->boundinfo))
- cur_index = partdesc->boundinfo->null_index;
- else if (!isnull[0])
- {
- /* Else bsearch in partdesc->boundinfo */
- bool equal = false;
+ else
+ {
+ bool equal = false;
+ int cur_offset;
- cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
- values, false, &equal);
- switch (key->strategy)
- {
- case PARTITION_STRATEGY_LIST:
+ /* 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:
+ case PARTITION_STRATEGY_RANGE:
+ {
+ bool equal = false;
+ int i;
+ int cur_offset;
/*
+ * 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;
+ }
+ }
+
+ /* 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
* found to be less or equal with the tuple. So, the bound
* 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-v13.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v13.patchDownload
From e536dcc54687db61aef1644bfb5898bb101cecd6 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Tue, 6 Jun 2017 11:31:34 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v13
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 | 70 +++-
src/backend/catalog/partition.c | 580 ++++++++++++++++++++++++++---
src/backend/commands/tablecmds.c | 46 ++-
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 | 25 +-
src/backend/utils/adt/ruleutils.c | 13 +
src/bin/psql/tab-complete.c | 2 +-
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 | 55 ++-
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 | 41 +-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
23 files changed, 1137 insertions(+), 88 deletions(-)
diff --git a/doc/src/sgml/ddl.sgml b/doc/src/sgml/ddl.sgml
index 84c4f20..5bc1a1b 100644
--- a/doc/src/sgml/ddl.sgml
+++ b/doc/src/sgml/ddl.sgml
@@ -2852,6 +2852,20 @@ VALUES ('Albany', NULL, NULL, 'NY');
<variablelist>
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
<listitem>
@@ -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 list, range, and hash.
</para>
<para>
@@ -3327,11 +3340,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 hash, 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.)
</para>
</listitem>
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index 56ea830..0468770 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 cities
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 0478e40..0fff4dd 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> ]
@@ -87,7 +87,8 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
<phrase>and <replaceable class="PARAMETER">partition_bound_spec</replaceable> is:</phrase>
{ IN ( { <replaceable class="PARAMETER">bound_literal</replaceable> | NULL } [, ...] ) |
- FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) }
+ FROM ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) TO ( { <replaceable class="PARAMETER">bound_literal</replaceable> | UNBOUNDED } [, ...] ) |
+ WITH ( MODULUS <replaceable class="PARAMETER">modulus</replaceable>, REMAINDER <replaceable class="PARAMETER">remainder</replaceable> ) }
<phrase><replaceable class="PARAMETER">index_parameters</replaceable> in <literal>UNIQUE</literal>, <literal>PRIMARY KEY</literal>, and <literal>EXCLUDE</literal> constraints are:</phrase>
@@ -301,6 +302,29 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -422,7 +446,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
</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
@@ -433,9 +457,16 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
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 btree operator class is specified when creating a
- partitioned table, the default btree 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 partitioning 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>
@@ -1636,6 +1667,29 @@ 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 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 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 8d03292..647b47a 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,18 +88,16 @@ typedef enum RangeDatumContent
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;
RangeDatumContent **content;/* what's contained in each range bound datum?
- * (see the above enum); NULL for list
- * partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * (see the above enum); 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 */
+ * if there isn't one, or is a hash or range
+ * partitioned table */
} PartitionBoundInfoData;
#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
@@ -97,6 +107,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
{
@@ -113,6 +131,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,
@@ -128,12 +147,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, RangeDatumContent *content1, 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 uint32 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
@@ -171,6 +197,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -236,7 +265,34 @@ 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;
@@ -463,6 +519,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));
@@ -610,53 +702,106 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->partnatts; j++)
+ if (b1->ndatums != b2->ndatums)
+ 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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -678,6 +823,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 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);
@@ -902,6 +1130,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);
@@ -1295,6 +1528,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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
@@ -1970,6 +2306,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2021,7 +2370,8 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key,
partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
+
/*
* Offset returned is such that the bound at offset is
* found to be less or equal with the tuple. So, the bound
@@ -2063,6 +2413,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
@@ -2239,6 +2621,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],
@@ -2322,3 +2713,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 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 b61fda9..f947575 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -459,7 +459,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,
@@ -833,7 +833,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);
@@ -13142,6 +13142,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
*strategy = PARTITION_STRATEGY_LIST;
else if (pg_strcasecmp(partspec->strategy, "range") == 0)
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (pg_strcasecmp(partspec->strategy, "hash") == 0)
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -13208,10 +13210,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)
@@ -13371,25 +13375,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 36bf1dc..8c9849b 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4444,6 +4444,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 5bcf031..852f6f6 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 c348bdc..a47ef23 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3543,6 +3543,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 81ddfc3..5a86ea0 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,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 ada95e5..8de3c44 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.
@@ -2651,8 +2652,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;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2677,6 +2730,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 9134fb9..af96f01 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3283,7 +3283,30 @@ 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")));
+
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("remainder for hash partition must be a non-negative integer")));
+
+ 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 6a0d273..1125016 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1652,6 +1652,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)
appendStringInfo(&buf, "LIST");
@@ -8649,6 +8653,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index d4b6976..80c60d7 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2060,7 +2060,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/pg_proc.h b/src/include/catalog/pg_proc.h
index 460cdb9..de09809 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5472,6 +5472,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 2d2e2c0..b1ccc4d 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -778,12 +778,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 */
+ /* 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 8aadbb8..8f3ab75 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,
@@ -3351,6 +3352,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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
--
@@ -3362,12 +3416,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;
@@ -3450,6 +3509,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(int4);
-- 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 5136506..2614f4a 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
@@ -521,6 +521,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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 (
@@ -528,6 +552,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 (
@@ -608,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, unbounded) TO (1, unbounded);
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,
@@ -756,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 8b0752a..97cbbca 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index c41b487..4534ed8 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 (
@@ -2180,6 +2181,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2191,12 +2248,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;
@@ -2258,6 +2319,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(int4);
-- 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 cb7aa5b..3b266cd 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 (
@@ -490,6 +487,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -497,6 +510,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
@@ -570,6 +584,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -639,6 +668,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 db8967b..7a147e4 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2
On Tue, Jun 6, 2017 at 1:03 PM, amul sul <sulamul@gmail.com> wrote:
May I ask you, how you sure about 8 is an unfit value for t1 relation?
And what if the value other than 8, for e.g. 7?
Well, First I created t1 as a leaf relation like below, and I tested
insert into t1 with value 8 and it was violating the partition
constraint of t1, however, 7 was fine.
create table t (a int) partition by hash(a);
create table t1 partition of t for values with (modulus 2, remainder 1);
Later I dropped this t1 and created 2 level partition with the leaf as a range.
drop table t1;
create table t1 partition of t for values with (modulus 2, remainder
1) partition by range(a);
create table t1_1 partition of t1 for values from (8) to (10);
So now, I am sure that t1_1 can accept the value 8 and its parent t1 can't.
So I think this can only happen in the case of partitioned by hash
that a value is legal for the child but illegal for the parent? Isn't
it a good idea that if a user is inserting in the top level relation
he should know for which partition exactly the constraint got
violated?
Updated patch attached.
Thanks.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 2017/06/06 17:50, Dilip Kumar wrote:
On Tue, Jun 6, 2017 at 1:03 PM, amul sul <sulamul@gmail.com> wrote:
May I ask you, how you sure about 8 is an unfit value for t1 relation?
And what if the value other than 8, for e.g. 7?Well, First I created t1 as a leaf relation like below, and I tested
insert into t1 with value 8 and it was violating the partition
constraint of t1, however, 7 was fine.create table t (a int) partition by hash(a);
create table t1 partition of t for values with (modulus 2, remainder 1);Later I dropped this t1 and created 2 level partition with the leaf as a range.
drop table t1;
create table t1 partition of t for values with (modulus 2, remainder
1) partition by range(a);
create table t1_1 partition of t1 for values from (8) to (10);So now, I am sure that t1_1 can accept the value 8 and its parent t1 can't.
So I think this can only happen in the case of partitioned by hash
that a value is legal for the child but illegal for the parent? Isn't
it a good idea that if a user is inserting in the top level relation
he should know for which partition exactly the constraint got
violated?
It's how the original partitioning code around ExecInsert/CopyFrom works,
not something that only affects hash partitioning. So, I think that
Amul's patch is fine and if we want to change something here, it should be
done by an independent patch. See the explanation below:
If we insert into a partition directly, we must check its partition
constraint. If the partition happens to be itself a partitioned table,
the constraint will be checked *after* tuple-routing and ExecConstraints()
is passed the leaf partition's ResultRelInfo, so if an error occurs there
we will use the leaf partition's name in the message. Since we combine
the leaf partition's own constraint with all of the ancestors' into a
single expression that is passed to ExecCheck(), it is hard to say exactly
which ancestor's constraint is violated. However, if the partition
constraint of some intervening ancestor had been violated, we wouldn't be
in ExecConstraints() at all; tuple-routing itself would have failed. So
it seems that we need worry (if at all) only about partition constraints
of the table mentioned in the insert statement.
Consider an example using the partition hierarchy:
root (a int, b char, c int) partition by range (a)
-> level1 from (1) to (10) partition by list (b)
-> level2 in ('a') parition by range (c)
-> leaf from (1) to (10)
Inserting (1, 'b', 1) into level1 will fail, because tuple can't be routed
at level1 (no partition defined for b = 'b').
Inserting (1, 'a', 10) into level1 will fail, because tuple can't be
routed at level2 (no partition defined for c >= 10).
Inserting (10, 'a', 1) into level1 will fail, because, although it was
able to get through level1 and level2 into leaf, a = 10 falls out of
level1's defined range. We don't check that 1 <= a < 10 before starting
the tuple-routing.
I wonder if we should... Since we don't allow BR triggers on partitioned
tables, there should not be any harm in doing it just before calling
ExecFindPartition(). Perhaps, topic for a new thread.
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 Tue, Jun 6, 2017 at 2:41 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
Consider an example using the partition hierarchy:
root (a int, b char, c int) partition by range (a)
-> level1 from (1) to (10) partition by list (b)
-> level2 in ('a') parition by range (c)
-> leaf from (1) to (10)
Inserting (1, 'b', 1) into level1 will fail, because tuple can't be routed
at level1 (no partition defined for b = 'b').Inserting (1, 'a', 10) into level1 will fail, because tuple can't be
routed at level2 (no partition defined for c >= 10).Inserting (10, 'a', 1) into level1 will fail, because, although it was
able to get through level1 and level2 into leaf, a = 10 falls out of
level1's defined range. We don't check that 1 <= a < 10 before starting
the tuple-routing.I wonder if we should... Since we don't allow BR triggers on partitioned
tables, there should not be any harm in doing it just before calling
ExecFindPartition(). Perhaps, topic for a new thread.
Yeah, correct.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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, 6 Jun 2017 13:03:58 +0530
amul sul <sulamul@gmail.com> wrote:
Updated patch attached.
I looked into the latest patch (v13) and have some comments
althogh they might be trivial.
First, I couldn't apply this patch to the latest HEAD due to
a documentation fix and pgintend updates. It needes rebase.
$ git apply /tmp/0002-hash-partitioning_another_design-v13.patch
error: patch failed: doc/src/sgml/ref/create_table.sgml:87
error: doc/src/sgml/ref/create_table.sgml: patch does not apply
error: patch failed: src/backend/catalog/partition.c:76
error: src/backend/catalog/partition.c: patch does not apply
error: patch failed: src/backend/commands/tablecmds.c:13371
error: src/backend/commands/tablecmds.c: patch does not apply
<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>
+
+ <varlistentry>
<term>Range Partitioning</term>
I think this section should be inserted after List Partitioning section because
the order of the descriptions is Range, List, then Hash in other places of
the documentation. At least,
- <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 list, range, and hash.
</para>
Also in this hunk. I think "The currently supported partitioning methods are
range, list, and hash." is better. We don't need to change the order of
the original description.
<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 hash, 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.)
Similarly, I think "Declarative partitioning only supports range, list and hash
partitioning," is better.
+
+ <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>
+
This paragraph should be inserted between "Create a list partitioned table:"
paragraph and "Ceate partition of a range partitioned table:" paragraph
as well as range and list.
*strategy = PARTITION_STRATEGY_LIST;
else if (pg_strcasecmp(partspec->strategy, "range") == 0)
*strategy = PARTITION_STRATEGY_RANGE;
+ else if (pg_strcasecmp(partspec->strategy, "hash") == 0)
+ *strategy = PARTITION_STRATEGY_HASH;
else
ereport(ERROR,
In the most of codes, the order is hash, range, then list, but only
in transformPartitionSpec(), the order is list, range, then hash,
as above. Maybe it is better to be uniform.
+ {
+ 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.")));
+
+
atttype,
- "btree",
- BTREE_AM_OID);
+ am_oid == HASH_AM_OID ? "hash" : "btree",
+ am_oid);
How about writing this part as following to reduce code redundancy?
+ Oid am_oid;
+ char *am_name;
<snip>
+ if (strategy == PARTITION_STRATEGY_HASH)
+ {
+ am_oid = HASH_AM_OID;
+ am_name = pstrdup("hash");
+ }
+ else
+ {
+ am_oid = BTREE_AM_OID;
+ am_name = pstrdup("btree");
+ }
+
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.")));
+ errmsg("data type %s has no default %s operator class",
+ format_type_be(atttype), am_name),
+ errhint("You must specify a %s operator class or define a default %s operator class for the data type.",
+ am_name, am_name)));
+
}
else
partopclass[attn] = ResolveOpClass(pelem->opclass,
atttype,
- "btree",
- BTREE_AM_OID);
+ am_name,
+ am_oid);
There is meaningless indentation change.
@@ -2021,7 +2370,8 @@ get_partition_for_tuple(PartitionDispatch *pd,
/* bsearch in partdesc->boundinfo */
cur_offset = partition_bound_bsearch(key,
partdesc->boundinfo,
- values, false, &equal);
+ values, false, &equal);
+
/*
* Offset returned is such that the bound at offset is
Fixing the comment of pg_get_partkeydef() is missing.
* pg_get_partkeydef
*
* Returns the partition key specification, ie, the following:
*
* PARTITION BY { RANGE | LIST } (column opt_collation opt_opclass [, ...])
*/
Datum
pg_get_partkeydef(PG_FUNCTION_ARGS)
{
Regards,
Regards,
Amul Sul
--
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
On Fri, 23 Jun 2017 13:41:15 +0900
Yugo Nagata <nagata@sraoss.co.jp> wrote:
On Tue, 6 Jun 2017 13:03:58 +0530
amul sul <sulamul@gmail.com> wrote:Updated patch attached.
I looked into the latest patch (v13) and have some comments
althogh they might be trivial.
One more comment:
+ if (spec->remainder < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+ errmsg("remainder for hash partition must be a non-negative integer")));
The value of remainder is defined as Iconst in gram.y, so it never be negative.
Hence, I think this check is not necessary or Assert is enough.
First, I couldn't apply this patch to the latest HEAD due to
a documentation fix and pgintend updates. It needes rebase.$ git apply /tmp/0002-hash-partitioning_another_design-v13.patch
error: patch failed: doc/src/sgml/ref/create_table.sgml:87
error: doc/src/sgml/ref/create_table.sgml: patch does not apply
error: patch failed: src/backend/catalog/partition.c:76
error: src/backend/catalog/partition.c: patch does not apply
error: patch failed: src/backend/commands/tablecmds.c:13371
error: src/backend/commands/tablecmds.c: patch does not apply<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> + + <varlistentry> <term>Range Partitioning</term>I think this section should be inserted after List Partitioning section because
the order of the descriptions is Range, List, then Hash in other places of
the documentation. At least,- <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 list, range, and hash. </para>Also in this hunk. I think "The currently supported partitioning methods are
range, list, and hash." is better. We don't need to change the order of
the original description.<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 hash, 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.)Similarly, I think "Declarative partitioning only supports range, list and hash
partitioning," is better.+ + <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> +This paragraph should be inserted between "Create a list partitioned table:"
paragraph and "Ceate partition of a range partitioned table:" paragraph
as well as range and list.*strategy = PARTITION_STRATEGY_LIST; else if (pg_strcasecmp(partspec->strategy, "range") == 0) *strategy = PARTITION_STRATEGY_RANGE; + else if (pg_strcasecmp(partspec->strategy, "hash") == 0) + *strategy = PARTITION_STRATEGY_HASH; else ereport(ERROR,In the most of codes, the order is hash, range, then list, but only
in transformPartitionSpec(), the order is list, range, then hash,
as above. Maybe it is better to be uniform.+ { + 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."))); + +atttype, - "btree", - BTREE_AM_OID); + am_oid == HASH_AM_OID ? "hash" : "btree", + am_oid);How about writing this part as following to reduce code redundancy?
+ Oid am_oid;
+ char *am_name;<snip>
+ if (strategy == PARTITION_STRATEGY_HASH) + { + am_oid = HASH_AM_OID; + am_name = pstrdup("hash"); + } + else + { + am_oid = BTREE_AM_OID; + am_name = pstrdup("btree"); + } + 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."))); + errmsg("data type %s has no default %s operator class", + format_type_be(atttype), am_name), + errhint("You must specify a %s operator class or define a default %s operator class for the data type.", + am_name, am_name))); + } else partopclass[attn] = ResolveOpClass(pelem->opclass, atttype, - "btree", - BTREE_AM_OID); + am_name, + am_oid);There is meaningless indentation change.
@@ -2021,7 +2370,8 @@ get_partition_for_tuple(PartitionDispatch *pd, /* bsearch in partdesc->boundinfo */ cur_offset = partition_bound_bsearch(key, partdesc->boundinfo, - values, false, &equal); + values, false, &equal); + /* * Offset returned is such that the bound at offset isFixing the comment of pg_get_partkeydef() is missing.
* pg_get_partkeydef
*
* Returns the partition key specification, ie, the following:
*
* PARTITION BY { RANGE | LIST } (column opt_collation opt_opclass [, ...])
*/
Datum
pg_get_partkeydef(PG_FUNCTION_ARGS)
{Regards,
Regards,
Amul Sul--
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
--
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
On Fri, Jun 23, 2017 at 10:11 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
On Tue, 6 Jun 2017 13:03:58 +0530
amul sul <sulamul@gmail.com> wrote:Updated patch attached.
I looked into the latest patch (v13) and have some comments
althogh they might be trivial.
Thanks for your review.
First, I couldn't apply this patch to the latest HEAD due to
a documentation fix and pgintend updates. It needes rebase.$ git apply /tmp/0002-hash-partitioning_another_design-v13.patch
error: patch failed: doc/src/sgml/ref/create_table.sgml:87
error: doc/src/sgml/ref/create_table.sgml: patch does not apply
error: patch failed: src/backend/catalog/partition.c:76
error: src/backend/catalog/partition.c: patch does not apply
error: patch failed: src/backend/commands/tablecmds.c:13371
error: src/backend/commands/tablecmds.c: patch does not apply
Fixed.
<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> + + <varlistentry> <term>Range Partitioning</term>I think this section should be inserted after List Partitioning section because
the order of the descriptions is Range, List, then Hash in other places of
the documentation. At least,
Fixed in the attached version.
- <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 list, range, and hash. </para>Also in this hunk. I think "The currently supported partitioning methods are
range, list, and hash." is better. We don't need to change the order of
the original description.
Fixed in the attached version.
<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 hash, 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.)Similarly, I think "Declarative partitioning only supports range, list and hash
partitioning," is better.
Fixed in the attached version.
+ + <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> +This paragraph should be inserted between "Create a list partitioned table:"
paragraph and "Ceate partition of a range partitioned table:" paragraph
as well as range and list.
Fixed in the attached version.
*strategy = PARTITION_STRATEGY_LIST; else if (pg_strcasecmp(partspec->strategy, "range") == 0) *strategy = PARTITION_STRATEGY_RANGE; + else if (pg_strcasecmp(partspec->strategy, "hash") == 0) + *strategy = PARTITION_STRATEGY_HASH; else ereport(ERROR,In the most of codes, the order is hash, range, then list, but only
in transformPartitionSpec(), the order is list, range, then hash,
as above. Maybe it is better to be uniform.
Make sense, fixed in the attached version.
+ { + 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."))); + +atttype, - "btree", - BTREE_AM_OID); + am_oid == HASH_AM_OID ? "hash" : "btree", + am_oid);How about writing this part as following to reduce code redundancy?
+ Oid am_oid;
+ char *am_name;<snip>
+ if (strategy == PARTITION_STRATEGY_HASH) + { + am_oid = HASH_AM_OID; + am_name = pstrdup("hash"); + } + else + { + am_oid = BTREE_AM_OID; + am_name = pstrdup("btree"); + } + 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."))); + errmsg("data type %s has no default %s operator class", + format_type_be(atttype), am_name), + errhint("You must specify a %s operator class or define a default %s operator class for the data type.", + am_name, am_name))); + } else partopclass[attn] = ResolveOpClass(pelem->opclass, atttype, - "btree", - BTREE_AM_OID); + am_name, + am_oid);
I had to have same thoughts before (see v12 patch & before), but
change due to review comments upthread.
There is meaningless indentation change.
@@ -2021,7 +2370,8 @@ get_partition_for_tuple(PartitionDispatch *pd, /* bsearch in partdesc->boundinfo */ cur_offset = partition_bound_bsearch(key, partdesc->boundinfo, - values, false, &equal); + values, false, &equal); + /* * Offset returned is such that the bound at offset is
Fixed in the attached version.
Fixing the comment of pg_get_partkeydef() is missing.
* pg_get_partkeydef
*
* Returns the partition key specification, ie, the following:
*
* PARTITION BY { RANGE | LIST } (column opt_collation opt_opclass [, ...])
*/
Datum
pg_get_partkeydef(PG_FUNCTION_ARGS)
{
Thanks to catching this, fixed in the attached version.
Regards,
Amul
Attachments:
0002-hash-partitioning_another_design-v14.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v14.patchDownload
From a9a8a8a2d561a93b891ecb8db770599b11759836 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 3 Jul 2017 15:49:52 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v14
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 | 70 +++-
src/backend/catalog/partition.c | 572 ++++++++++++++++++++++++++---
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/bin/psql/tab-complete.c | 2 +-
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 | 55 ++-
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 | 41 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
23 files changed, 1131 insertions(+), 87 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 b15c19d..c2ce58c 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> | UNBOUNDED } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | UNBOUNDED } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | UNBOUNDED } [, ...] ) |
+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>
@@ -310,6 +311,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
@@ -432,7 +456,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
@@ -443,9 +467,16 @@ 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 partitioning 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>
@@ -1596,6 +1627,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
@@ -1646,6 +1687,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 a7efe7e..ea830cd 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,16 +88,13 @@ typedef enum RangeDatumContent
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;
RangeDatumContent **content; /* what's contained in each range bound
* datum? (see the above enum); NULL for
- * list partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * hasn and list partitioned tables */
+ int *indexes; /* Partition indexes */
int null_index; /* Index of the null-accepting partition; -1
* if there isn't one */
} PartitionBoundInfoData;
@@ -97,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
{
@@ -113,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,
@@ -128,12 +146,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -147,6 +168,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 uint32 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
@@ -171,6 +196,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -236,7 +264,34 @@ 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;
@@ -464,6 +519,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));
@@ -609,53 +700,106 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->partnatts; j++)
+ if (b1->ndatums != b2->ndatums)
+ 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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -677,6 +821,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 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);
@@ -901,6 +1128,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);
@@ -1294,6 +1526,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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
@@ -1994,6 +2329,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2091,6 +2439,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
@@ -2267,6 +2647,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],
@@ -2350,3 +2739,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 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 bb00858..670cf62 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,
@@ -842,7 +842,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);
@@ -13154,7 +13154,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;
@@ -13224,10 +13226,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)
@@ -13387,25 +13391,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 67ac814..a0b4955 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4445,6 +4445,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 91d64b7..f6b753f 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 3a23f0b..66dab4a 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3544,6 +3544,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 2988e8b..a2b6569 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,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 0f3998f..eda9999 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.
@@ -2651,8 +2652,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;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2677,6 +2730,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 ee5f3a3..3eba75e 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3303,7 +3303,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 18d9e27..dd47c18 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1548,7 +1548,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)
@@ -1652,6 +1652,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)
appendStringInfo(&buf, "LIST");
@@ -8649,6 +8653,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 17344d9..4cba9da 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2072,7 +2072,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/pg_proc.h b/src/include/catalog/pg_proc.h
index 8b33b4e..5b757f4 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5475,6 +5475,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 1d96169..4a96d82 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 13d6a4b..7ebc144 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,
@@ -3357,6 +3358,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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
--
@@ -3368,12 +3422,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;
@@ -3469,6 +3528,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(int4);
-- 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 fb8745b..c8eb73f 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
@@ -521,6 +521,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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 (
@@ -528,6 +552,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 (
@@ -608,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, unbounded) TO (1, unbounded);
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,
@@ -756,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 d1153f4..3a17d01 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 5dd1402..19dc1b0 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 (
@@ -2185,6 +2186,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2196,12 +2253,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;
@@ -2273,6 +2334,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(int4);
-- 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 cb7aa5b..3b266cd 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 (
@@ -490,6 +487,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -497,6 +510,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
@@ -570,6 +584,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -639,6 +668,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 83c3ad8..8cded87 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2
0001-Cleanup_v6.patchapplication/octet-stream; name=0001-Cleanup_v6.patchDownload
From 5147b561e3df1347a47e52831185c7132fa21722 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 v14.
---
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 7da2058..a7efe7e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1942,9 +1942,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;
@@ -1988,48 +1986,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:
+ 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;
+ }
+ }
+
+ /* 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
@@ -2037,12 +2056,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
On Fri, Jun 23, 2017 at 11:19 AM, Yugo Nagata <nagata@sraoss.co.jp> wrote:
On Fri, 23 Jun 2017 13:41:15 +0900
Yugo Nagata <nagata@sraoss.co.jp> wrote:On Tue, 6 Jun 2017 13:03:58 +0530
amul sul <sulamul@gmail.com> wrote:Updated patch attached.
I looked into the latest patch (v13) and have some comments
althogh they might be trivial.One more comment:
+ if (spec->remainder < 0) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TABLE_DEFINITION), + errmsg("remainder for hash partition must be a non-negative integer")));The value of remainder is defined as Iconst in gram.y, so it never be negative.
Hence, I think this check is not necessary or Assert is enough.
Make sense, fixed this as well in the v14 patch. Thanks again.
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 Mon, Jul 3, 2017 at 4:39 PM, amul sul <sulamul@gmail.com> wrote:
Thanks to catching this, fixed in the attached version.
Few comments on the latest version.
0001 looks fine, for 0002 I have some comments.
1.
+ hbounds = (PartitionHashBound * *) palloc(nparts *
+ sizeof(PartitionHashBound *));
/s/(PartitionHashBound * *)/(PartitionHashBound **)/g
2.
RelationBuildPartitionDesc
{
....
* catalog scan that retrieved them, whereas that in the latter is
* defined by canonicalized representation of the list values or the
* range bounds.
*/
for (i = 0; i < nparts; i++)
result->oids[mapping[i]] = oids[i];
Should this comments mention about hash as well?
3.
if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
return false;
if (b1->ndatums != b2->ndatums)
return false;
If ndatums itself is different then no need to access datum memory, so
better to check ndatum first.
4.
+ * next larger modulus. For example, if you have a bunch
+ * of partitions that all have modulus 5, you can add a
+ * new new partition with modulus 10 or a new partition
Typo, "new new partition" -> "new partition"
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.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 Wed, Jul 5, 2017 at 4:50 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Jul 3, 2017 at 4:39 PM, amul sul <sulamul@gmail.com> wrote:
Thanks to catching this, fixed in the attached version.
Few comments on the latest version.
Thanks for your review, please find my comment inline:
0001 looks fine, for 0002 I have some comments.
1. + hbounds = (PartitionHashBound * *) palloc(nparts * + sizeof(PartitionHashBound *));/s/(PartitionHashBound * *)/(PartitionHashBound **)/g
Fixed in the attached version.
2.
RelationBuildPartitionDesc
{
....* catalog scan that retrieved them, whereas that in the latter is
* defined by canonicalized representation of the list values or the
* range bounds.
*/
for (i = 0; i < nparts; i++)
result->oids[mapping[i]] = oids[i];Should this comments mention about hash as well?
Instead, I have generalised this comment in the attached patch
3.
if (b1->datums[b1->ndatums - 1][0] != b2->datums[b2->ndatums - 1][0])
return false;if (b1->ndatums != b2->ndatums)
return false;If ndatums itself is different then no need to access datum memory, so
better to check ndatum first.
You are correct, we already doing this in the
partition_bounds_equal(). This is a redundant code, removed in the
attached version.
4. + * next larger modulus. For example, if you have a bunch + * of partitions that all have modulus 5, you can add a + * new new partition with modulus 10 or a new partitionTypo, "new new partition" -> "new partition"
Fixed in the attached version.
Regards,
Amul
Attachments:
0001-Cleanup_v6.patchapplication/octet-stream; name=0001-Cleanup_v6.patchDownload
From e072a543f8ed3c53013d3c8df4d0f3cca03f13a0 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 v14.
---
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 7da2058..a7efe7e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1942,9 +1942,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;
@@ -1988,48 +1986,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:
+ 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;
+ }
+ }
+
+ /* 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
@@ -2037,12 +2056,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-v15.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v15.patchDownload
From 0b3250362c5cc9b0dff5b7e06902ec2db8095a9d Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Mon, 3 Jul 2017 15:49:52 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v15
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 | 70 +++-
src/backend/catalog/partition.c | 572 ++++++++++++++++++++++++++---
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/bin/psql/tab-complete.c | 2 +-
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 | 55 ++-
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 | 41 ++-
src/test/regress/sql/insert.sql | 33 ++
src/test/regress/sql/update.sql | 28 ++
23 files changed, 1129 insertions(+), 89 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 b15c19d..c2ce58c 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> | UNBOUNDED } [, ...] )
- TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | UNBOUNDED } [, ...] )
+ TO ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replaceable class="PARAMETER">string_literal</replaceable> | UNBOUNDED } [, ...] ) |
+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>
@@ -310,6 +311,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
@@ -432,7 +456,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
@@ -443,9 +467,16 @@ 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 partitioning 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>
@@ -1596,6 +1627,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
@@ -1646,6 +1687,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 a7efe7e..b1c9071 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,12 +58,24 @@
* 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.
+ * 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.
*/
/* Ternary value to represent what's contained in a range bound datum */
@@ -76,16 +88,13 @@ typedef enum RangeDatumContent
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;
RangeDatumContent **content; /* what's contained in each range bound
* datum? (see the above enum); NULL for
- * list partitioned tables */
- int *indexes; /* Partition indexes; one entry per member of
- * the datums array (plus one if range
- * partitioned table) */
+ * hasn and list partitioned tables */
+ int *indexes; /* Partition indexes */
int null_index; /* Index of the null-accepting partition; -1
* if there isn't one */
} PartitionBoundInfoData;
@@ -97,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
{
@@ -113,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,
@@ -128,12 +146,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, RangeDatumContent *content1, bool lower1,
PartitionRangeBound *b2);
@@ -147,6 +168,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 uint32 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
@@ -171,6 +196,9 @@ RelationBuildPartitionDesc(Relation rel)
int ndatums = 0;
+ /* Hash partitioning specific */
+ PartitionHashBound **hbounds = NULL;
+
/* List partitioning specific */
PartitionListValue **all_values = NULL;
int null_index = -1;
@@ -236,7 +264,34 @@ 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;
@@ -464,6 +519,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));
@@ -574,8 +665,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];
@@ -609,53 +699,103 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->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->content != NULL)
+ int j;
+
+ for (j = 0; j < key->partnatts; j++)
{
+ /* For range partitions, the bounds might not be finite. */
+ if (b1->content != NULL)
+ {
+ /*
+ * A finite bound always differs from an infinite bound,
+ * and different kinds of infinities differ from each
+ * other.
+ */
+ if (b1->content[i][j] != b2->content[i][j])
+ return false;
+
+ /*
+ * Non-finite bounds are equal without further
+ * examination.
+ */
+ if (b1->content[i][j] != RANGE_DATUM_FINITE)
+ continue;
+ }
+
/*
- * A finite bound always differs from an infinite bound, and
- * different kinds of infinities differ from each other.
+ * 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 (b1->content[i][j] != b2->content[i][j])
+ if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+ key->parttypbyval[j],
+ key->parttyplen[j]))
return false;
-
- /* Non-finite bounds are equal without further examination. */
- if (b1->content[i][j] != RANGE_DATUM_FINITE)
- 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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -677,6 +817,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);
@@ -901,6 +1124,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);
@@ -1294,6 +1522,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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
@@ -1994,6 +2325,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2091,6 +2435,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
@@ -2267,6 +2643,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],
@@ -2350,3 +2735,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 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 bb00858..670cf62 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,
@@ -842,7 +842,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);
@@ -13154,7 +13154,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;
@@ -13224,10 +13226,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)
@@ -13387,25 +13391,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 67ac814..a0b4955 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4445,6 +4445,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 91d64b7..f6b753f 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 3a23f0b..66dab4a 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3544,6 +3544,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 2988e8b..a2b6569 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,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 0f3998f..eda9999 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.
@@ -2651,8 +2652,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;
+
+ $$ = (Node *) n;
+ }
/* a LIST partition */
- FOR VALUES IN_P '(' partbound_datum_list ')'
+ | FOR VALUES IN_P '(' partbound_datum_list ')'
{
PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
@@ -2677,6 +2730,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 ee5f3a3..3eba75e 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3303,7 +3303,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 18d9e27..dd47c18 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1548,7 +1548,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)
@@ -1652,6 +1652,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)
appendStringInfo(&buf, "LIST");
@@ -8649,6 +8653,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 17344d9..4cba9da 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2072,7 +2072,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/pg_proc.h b/src/include/catalog/pg_proc.h
index 8b33b4e..5b757f4 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5475,6 +5475,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 1d96169..4a96d82 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 13d6a4b..7ebc144 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,
@@ -3357,6 +3358,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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
--
@@ -3368,12 +3422,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;
@@ -3469,6 +3528,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(int4);
-- 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 fb8745b..c8eb73f 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
@@ -521,6 +521,30 @@ ERROR: cannot specify finite value after UNBOUNDED
LINE 1: ...ge_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNB...
^
DROP TABLE range_parted_multicol;
+-- 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 (
@@ -528,6 +552,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 (
@@ -608,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, unbounded) TO (1, unbounded);
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,
@@ -756,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 d1153f4..3a17d01 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 5dd1402..19dc1b0 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 (
@@ -2185,6 +2186,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2196,12 +2253,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;
@@ -2273,6 +2334,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(int4);
-- 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 cb7aa5b..3b266cd 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 (
@@ -490,6 +487,22 @@ CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a,
CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
DROP TABLE range_parted_multicol;
+-- 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
@@ -497,6 +510,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
@@ -570,6 +584,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, unbounded) TO (1, unbounded);
+-- 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 (
@@ -639,6 +668,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 83c3ad8..8cded87 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2
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 07dcf1c71ec07f0e42ffd2bd2ddd701a4b7eef38 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 v16.
---
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 e20ddce..1e9ac39 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1923,9 +1923,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;
@@ -1969,48 +1967,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
@@ -2018,12 +2037,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-v16.patchapplication/octet-stream; name=0002-hash-partitioning_another_design-v16.patchDownload
From aeda88b72f3655053e5b6a69fbb49e32f1825a11 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Thu, 27 Jul 2017 17:03:44 +0530
Subject: [PATCH 2/2] hash-partitioning_another_design-v16
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 | 70 +++-
src/backend/catalog/partition.c | 566 ++++++++++++++++++++++++++---
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/bin/psql/tab-complete.c | 2 +-
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 ++
23 files changed, 1130 insertions(+), 89 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..4ecbd9c 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,16 @@ 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 partitioning 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>
@@ -1629,6 +1660,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 +1720,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 1e9ac39..34a6978 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -58,26 +58,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 +98,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 +122,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 +138,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 +160,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 uint32 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 +188,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 +256,34 @@ 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 +507,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 +653,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,50 +687,94 @@ partition_bounds_equal(PartitionKey key,
if (b1->null_index != b2->null_index)
return false;
- for (i = 0; i < b1->ndatums; i++)
+ if (key->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 < key->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 < key->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],
+ key->parttypbyval[j],
+ key->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],
- key->parttypbyval[j],
- key->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 (key->strategy == PARTITION_STRATEGY_RANGE &&
+ b1->indexes[i] != b2->indexes[i])
return false;
}
- /* There are ndatums+1 indexes in case of range partitions */
- if (key->strategy == PARTITION_STRATEGY_RANGE &&
- b1->indexes[i] != b2->indexes[i])
- return false;
-
return true;
}
@@ -662,6 +796,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);
@@ -873,6 +1090,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);
@@ -1267,6 +1489,109 @@ 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(a), hash_fn_2(b))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1(a), hash_fn_2(b))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1(a), hash_fn_2(b))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1(a), hash_fn_2(b))
+ *
+ * where hash_fn_1 and hash_fn_2 are be datatype-specific hash functions for
+ * columns a and b respectively.
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+ FuncExpr *fexpr;
+ Node *modulusConst;
+ Node *remainderConst;
+ 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);
+
+ 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_make1(keyCol),
+ 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
@@ -1975,6 +2300,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);
+ uint32 rowHash = compute_hash_value(key, values,
+ isnull);
+
+ cur_index = boundinfo->indexes[rowHash % greatest_modulus];
+ }
+ break;
+
case PARTITION_STRATEGY_LIST:
/*
@@ -2072,6 +2410,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
@@ -2255,6 +2625,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],
@@ -2338,3 +2717,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 32-bit value.
+ */
+static uint32
+mix_hash_value(int nkeys, uint32 *hash_array, bool *isnull)
+{
+ int i;
+ uint32 rowHash = 0;
+
+ for (i = 0; i < nkeys; i++)
+ {
+ /*
+ * Like TupleHashTableHash, rotate hashkey left 1 bit at each step.
+ * This prevents equal values in different keys from cancelling each
+ * other.
+ */
+ rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+ if (!isnull[i])
+ rowHash ^= hash_array[i];
+ }
+
+ return rowHash;
+}
+
+/*
+ * compute_hash_value
+ *
+ * Compute the hash value for given not null partition key values.
+ */
+static uint32
+compute_hash_value(PartitionKey key, Datum *values, bool *isnull)
+{
+ int i;
+ int nkeys = key->partnatts;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+
+ 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] = DatumGetUInt32(FunctionCall1(&key->partsupfunc[i],
+ values[i]));
+ }
+ }
+
+ /* Form a single 32-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;
+ uint32 hash_array[PARTITION_MAX_KEYS];
+ bool isnull[PARTITION_MAX_KEYS];
+ uint32 rowHash = 0;
+
+ 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_UINT32(i + 2);
+ }
+
+ /* Form a single 32-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 bb00858..670cf62 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,
@@ -842,7 +842,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);
@@ -13154,7 +13154,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;
@@ -13224,10 +13226,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)
@@ -13387,25 +13391,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 45a04b0..ef482ab 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4445,6 +4445,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 379d92a..04eabc6 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3571,6 +3571,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 86c811d..01c1763 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2388,6 +2388,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 4b1ce09..5cb3077 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.
@@ -2651,8 +2652,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);
@@ -2677,6 +2730,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 9f37f1b..0163f1b 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3303,7 +3303,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 d83377d..c06b4b4 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)
appendStringInfo(&buf, "LIST");
@@ -8699,6 +8703,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/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index e9fdc90..cbf7905 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/pg_proc.h b/src/include/catalog/pg_proc.h
index 8b33b4e..5b757f4 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5475,6 +5475,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 13d6a4b..7ebc144 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,
@@ -3357,6 +3358,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+-- 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
--
@@ -3368,12 +3422,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;
@@ -3469,6 +3528,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(int4);
-- 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 aa44c11..f8bafbe 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 (
@@ -601,6 +632,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,
@@ -749,6 +797,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 0dcc86f..d3d6d1e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+ERROR: new row for relation "hpart1" violates partition constraint
+DETAIL: Failing row contains (11).
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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
+--------+----+-------------------
+ hpart1 | 4 | 0
+ hpart1 | 8 | 0
+ hpart1 | 12 | 0
+ hpart1 | 16 | 0
+ hpart2 | 1 | 1
+ hpart2 | 5 | 1
+ hpart2 | 9 | 1
+ hpart3 | 2 | 2
+ hpart3 | 6 | 2
+ hpart3 | 10 | 2
+ hpart4 | 3 | 3
+ hpart4 | 7 | 3
+ hpart4 | 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(int4);
-- 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..55fbc9e 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+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(int4);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 5dd1402..19dc1b0 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 (
@@ -2185,6 +2186,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) RETURNS int4 AS
+$$ BEGIN RETURN a; END; $$ LANGUAGE 'plpgsql' IMMUTABLE;
+CREATE OPERATOR CLASS custom_opclass FOR TYPE int4 USING HASH AS
+OPERATOR 1 = , FUNCTION 1 dummy_hashint4(int4);
+
+-- 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
--
@@ -2196,12 +2253,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;
@@ -2273,6 +2334,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(int4);
-- 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 1c0ce92..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 6adf25d..af9bd75 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+create table hash_parted (
+ a int
+) partition by hash (a custom_opclass);
+create table hpart1 partition of hash_parted for values with (modulus 4, remainder 0);
+create table hpart2 partition of hash_parted for values with (modulus 4, remainder 1);
+create table hpart3 partition of hash_parted for values with (modulus 4, remainder 2);
+create table hpart4 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 hpart1 values(12),(16);
+-- fail;
+insert into hpart1 values(11);
+-- 11 % 4 -> 3 remainder i.e. valid data for hpart4 partition
+insert into hpart4 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(int4);
-- 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..873801b 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) returns int4 as
+$$ begin return a; end; $$ language 'plpgsql' immutable;
+create operator class custom_opclass for type int4 using hash as
+operator 1 = , function 1 dummy_hashint4(int4);
+
+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(int4);
--
2.6.2