Thread: [HACKERS] [POC] hash partitioning

[HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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>

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Aleksander Alekseev
Date:
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]. 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 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>

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

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

Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
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





Re: [HACKERS] [POC] hash partitioning

From
Rushabh Lathia
Date:


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


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

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 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.
>
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. ​

​+1 for the default hash function corresponding to partitioning key type.​

Regards,
Amul




Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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 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>
> 
> > 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;
> >  
> 
> > 
> > -- 
> > 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>



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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 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.

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>



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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 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)
> >
> >
> 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-hackers
> >
> >
> 
> 
> Regards,
> 
> Rushabh Lathia


-- 
Yugo Nagata <nagata@sraoss.co.jp>



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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 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.
> >
> ​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>



Re: [HACKERS] [POC] hash partitioning

From
Aleksander Alekseev
Date:
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
expressionsin 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]:

```
***************
*** 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
CREATETABLE partitioned (     a int 
```

Exact script I'm using for building and testing PostgreSQL could be
found here [2].

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

Re: [HACKERS] [POC] hash partitioning

From
Maksim Milyutin
Date:
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 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'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



Re: [HACKERS] [POC] hash partitioning

From
Aleksander Alekseev
Date:
> 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 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'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

Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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.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 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>
> > > 
> > > > 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;
> > > >  
> > > 
> > > > 
> > > > -- 
> > > > 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>



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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?

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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 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?


--
Yugo Nagata <nagata@sraoss.co.jp>



Re: [HACKERS] [POC] hash partitioning

From
Greg Stark
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 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.

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​

Re: [HACKERS] [POC] hash partitioning

From
David Steele
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
David Steele
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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>

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [POC] hash partitioning

From
Tatsuo Ishii
Date:
> 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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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,
remainder0);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>



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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>



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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
http://postgr.es/m/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



Re: [HACKERS] [POC] hash partitioning

From
Jeff Davis
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Jeff Davis
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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 http://postgr.es/m/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



Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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 *
partitionedtables */
 

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; /*
Indexof 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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
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.

Thanks,
Amit

[1]
https://www.postgresql.org/message-id/CA%2BTgmoZN_Zf7MBb48O66FAJgFe0S9_NkLVeQNBz6hsxb6Og93w%40mail.gmail.com




Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
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




Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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
nullsin 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



Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
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




Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 <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).
>
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 partitioning
>
> This 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 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
>
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 http://postgr.es/m/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]. https://www.postgresql.org/message-id/CA%2BTgmoaE9NZ_RiqZQLp2aJXPO4E78QxkQYL-FR2zCDop96Ahdg%40mail.gmail.com

Regards,
Amul Sul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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; -1
>
> Comments 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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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 <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).

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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 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.
>
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 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)
>
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] https://www.postgresql.org/message-id/CA%2BTgmoaE9NZ_RiqZQLp2aJXPO4E78QxkQYL-FR2zCDop96Ahdg%40mail.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

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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 1
>
> Seems 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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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 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.
>>
> 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        *
preventsequal 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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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 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.
>>>
>> 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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Peter Eisentraut
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Peter Eisentraut
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 of
>
> Wouldn'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, because
>
> fails, 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 partition
>
> I 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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Tom Lane
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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 of
>>
>> Wouldn'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, because
>>
>> fails, 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 partition
>>
>> I 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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 of
>>>
>>> Wouldn'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, because
>>>
>>> fails, 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 partition
>>>
>>> I 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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
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




Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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
forwhich 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



Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
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




Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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
forthe 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
forthe 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
thedata 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
datatype.",
 
+                                 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 }
(columnopt_collation opt_opclass [, ...])*/
 
Datum
pg_get_partkeydef(PG_FUNCTION_ARGS)
{

Regards,

> 
> Regards,
> Amul Sul


-- 
Yugo Nagata <nagata@sraoss.co.jp>



Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
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
forthe 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
classfor 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
forthe 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
datatype.",
 
> +                                 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


-- 
Yugo Nagata <nagata@sraoss.co.jp>



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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
defaulthash 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
defaultbtree 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
btreeoperator 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
operatorclass 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

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
Dilip Kumar
Date:
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



Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
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 partition
>
> Typo, "new new partition"  -> "new partition"
>

Fixed in the attached version.

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

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
Attaching newer patches rebased against the latest master head. Thanks !

Regards,
Amul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
"yangjie@highgo.com"
Date:
Hello

Looking at your hash partitioning syntax, I implemented a hash partition in a more concise way, with no need to determine the number of sub-tables, and dynamically add partitions.

Description

The hash partition's implement is on the basis of the original range / list partition,and using similar syntax.

To create a partitioned table ,use:

CREATE TABLE h (id int) PARTITION BY HASH(id);

The partitioning key supports only one value, and I think the partition key can support multiple values, 
which may be difficult to implement when querying, but it is not impossible.

A partition table can be create as bellow:

 CREATE TABLE h1 PARTITION OF h;
 CREATE TABLE h2 PARTITION OF h;
 CREATE TABLE h3 PARTITION OF h;
 
FOR VALUES clause cannot be used, and the partition bound is calclulated automatically as partition index of single integer value.

An inserted record is stored in a partition whose index equals 
DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, values[0])) % nparts
/* Number of partitions */
;
In the above example, this is DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, id)) % 3;

postgres=# insert into h select generate_series(1,20);
INSERT 0 20
postgres=# select tableoid::regclass,* from h;
 tableoid | id 
----------+----
 h1       |  3
 h1       |  5
 h1       | 17
 h1       | 19
 h2       |  2
 h2       |  6
 h2       |  7
 h2       | 11
 h2       | 12
 h2       | 14
 h2       | 15
 h2       | 18
 h2       | 20
 h3       |  1
 h3       |  4
 h3       |  8
 h3       |  9
 h3       | 10
 h3       | 13
 h3       | 16
(20 rows)

The number of partitions here can be dynamically added, and if a new partition is created, the number of partitions changes, the calculated target partitions will change, and the same data is not reasonable in different partitions,So you need to re-calculate the existing data and insert the target partition when you create a new partition.

postgres=# create table h4 partition of h;
CREATE TABLE
postgres=# select tableoid::regclass,* from h;
 tableoid | id 
----------+----
 h1       |  5
 h1       | 17
 h1       | 19
 h1       |  6
 h1       | 12
 h1       |  8
 h1       | 13
 h2       | 11
 h2       | 14
 h3       |  1
 h3       |  9
 h3       |  2
 h3       | 15
 h4       |  3
 h4       |  7
 h4       | 18
 h4       | 20
 h4       |  4
 h4       | 10
 h4       | 16
(20 rows)

When querying the data, the hash partition uses the same algorithm as the insertion, and filters out the table that does not need to be scanned.

postgres=# explain analyze select * from h where id = 1;
                                             QUERY PLAN                                             
----------------------------------------------------------------------------------------------------
 Append  (cost=0.00..41.88 rows=13 width=4) (actual time=0.020..0.023 rows=1 loops=1)
   ->  Seq Scan on h3  (cost=0.00..41.88 rows=13 width=4) (actual time=0.013..0.016 rows=1 loops=1)
         Filter: (id = 1)
         Rows Removed by Filter: 3
 Planning time: 0.346 ms
 Execution time: 0.061 ms
(6 rows)

postgres=# explain analyze select * from h where id in (1,5);;
                                             QUERY PLAN                                             
----------------------------------------------------------------------------------------------------
 Append  (cost=0.00..83.75 rows=52 width=4) (actual time=0.016..0.028 rows=2 loops=1)
   ->  Seq Scan on h1  (cost=0.00..41.88 rows=26 width=4) (actual time=0.015..0.018 rows=1 loops=1)
         Filter: (id = ANY ('{1,5}'::integer[]))
         Rows Removed by Filter: 6
   ->  Seq Scan on h3  (cost=0.00..41.88 rows=26 width=4) (actual time=0.005..0.007 rows=1 loops=1)
         Filter: (id = ANY ('{1,5}'::integer[]))
         Rows Removed by Filter: 3
 Planning time: 0.720 ms
 Execution time: 0.074 ms
(9 rows)

postgres=# explain analyze select * from h where id = 1 or id = 5;;
                                             QUERY PLAN                                             
----------------------------------------------------------------------------------------------------
 Append  (cost=0.00..96.50 rows=50 width=4) (actual time=0.017..0.078 rows=2 loops=1)
   ->  Seq Scan on h1  (cost=0.00..48.25 rows=25 width=4) (actual time=0.015..0.019 rows=1 loops=1)
         Filter: ((id = 1) OR (id = 5))
         Rows Removed by Filter: 6
   ->  Seq Scan on h3  (cost=0.00..48.25 rows=25 width=4) (actual time=0.005..0.010 rows=1 loops=1)
         Filter: ((id = 1) OR (id = 5))
         Rows Removed by Filter: 3
 Planning time: 0.396 ms
 Execution time: 0.139 ms
(9 rows)

Can not detach / attach / drop partition table.

Best regards,
young

yonj1e.github.io

yangjie@highgo.com

Re: [HACKERS] [POC] hash partitioning

From
Yugo Nagata
Date:
Hi young,

On Mon, 28 Aug 2017 15:33:46 +0800
"yangjie@highgo.com" <yangjie@highgo.com> wrote:

> Hello
> 
> Looking at your hash partitioning syntax, I implemented a hash partition in a more concise way, with no need to
determinethe number of sub-tables, and dynamically add partitions.
 

I think it is great work, but the current consensus about hash-partitioning supports 
Amul's patch[1], in which the syntax is different from the my original proposal. 
So, you will have to read Amul's patch and make a discussion if you still want to
propose your implementation.

Regards,

[1] https://www.postgresql.org/message-id/CAAJ_b965A2oog=6eFUhELexL3RmgFssB3G7LwkVA1bw0WUJJoA@mail.gmail.com


> 
> Description
> 
> The hash partition's implement is on the basis of the original range / list partition,and using similar syntax.
> 
> To create a partitioned table ,use:
> 
> CREATE TABLE h (id int) PARTITION BY HASH(id);
> 
> The partitioning key supports only one value, and I think the partition key can support multiple values, 
> which may be difficult to implement when querying, but it is not impossible.
> 
> A partition table can be create as bellow:
> 
>  CREATE TABLE h1 PARTITION OF h;
>  CREATE TABLE h2 PARTITION OF h;
>  CREATE TABLE h3 PARTITION OF h;
>  
> FOR VALUES clause cannot be used, and the partition bound is calclulated automatically as partition index of single
integervalue.
 
> 
> An inserted record is stored in a partition whose index equals 
> DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, values[0])) %
nparts/*Number of partitions */
 
> ;
> In the above example, this is DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0],
TYPECACHE_HASH_PROC)->hash_proc,id)) % 3;
 
> 
> postgres=# insert into h select generate_series(1,20);
> INSERT 0 20
> postgres=# select tableoid::regclass,* from h;
>  tableoid | id 
> ----------+----
>  h1       |  3
>  h1       |  5
>  h1       | 17
>  h1       | 19
>  h2       |  2
>  h2       |  6
>  h2       |  7
>  h2       | 11
>  h2       | 12
>  h2       | 14
>  h2       | 15
>  h2       | 18
>  h2       | 20
>  h3       |  1
>  h3       |  4
>  h3       |  8
>  h3       |  9
>  h3       | 10
>  h3       | 13
>  h3       | 16
> (20 rows)
> 
> The number of partitions here can be dynamically added, and if a new partition is created, the number of partitions
changes,the calculated target partitions will change, and the same data is not reasonable in different partitions,So
youneed to re-calculate the existing data and insert the target partition when you create a new partition.
 
> 
> postgres=# create table h4 partition of h;
> CREATE TABLE
> postgres=# select tableoid::regclass,* from h;
>  tableoid | id 
> ----------+----
>  h1       |  5
>  h1       | 17
>  h1       | 19
>  h1       |  6
>  h1       | 12
>  h1       |  8
>  h1       | 13
>  h2       | 11
>  h2       | 14
>  h3       |  1
>  h3       |  9
>  h3       |  2
>  h3       | 15
>  h4       |  3
>  h4       |  7
>  h4       | 18
>  h4       | 20
>  h4       |  4
>  h4       | 10
>  h4       | 16
> (20 rows)
> 
> When querying the data, the hash partition uses the same algorithm as the insertion, and filters out the table that
doesnot need to be scanned.
 
> 
> postgres=# explain analyze select * from h where id = 1;
>                                              QUERY PLAN                                             
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..41.88 rows=13 width=4) (actual time=0.020..0.023 rows=1 loops=1)
>    ->  Seq Scan on h3  (cost=0.00..41.88 rows=13 width=4) (actual time=0.013..0.016 rows=1 loops=1)
>          Filter: (id = 1)
>          Rows Removed by Filter: 3
>  Planning time: 0.346 ms
>  Execution time: 0.061 ms
> (6 rows)
> 
> postgres=# explain analyze select * from h where id in (1,5);;
>                                              QUERY PLAN                                             
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..83.75 rows=52 width=4) (actual time=0.016..0.028 rows=2 loops=1)
>    ->  Seq Scan on h1  (cost=0.00..41.88 rows=26 width=4) (actual time=0.015..0.018 rows=1 loops=1)
>          Filter: (id = ANY ('{1,5}'::integer[]))
>          Rows Removed by Filter: 6
>    ->  Seq Scan on h3  (cost=0.00..41.88 rows=26 width=4) (actual time=0.005..0.007 rows=1 loops=1)
>          Filter: (id = ANY ('{1,5}'::integer[]))
>          Rows Removed by Filter: 3
>  Planning time: 0.720 ms
>  Execution time: 0.074 ms
> (9 rows)
> 
> postgres=# explain analyze select * from h where id = 1 or id = 5;;
>                                              QUERY PLAN                                             
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..96.50 rows=50 width=4) (actual time=0.017..0.078 rows=2 loops=1)
>    ->  Seq Scan on h1  (cost=0.00..48.25 rows=25 width=4) (actual time=0.015..0.019 rows=1 loops=1)
>          Filter: ((id = 1) OR (id = 5))
>          Rows Removed by Filter: 6
>    ->  Seq Scan on h3  (cost=0.00..48.25 rows=25 width=4) (actual time=0.005..0.010 rows=1 loops=1)
>          Filter: ((id = 1) OR (id = 5))
>          Rows Removed by Filter: 3
>  Planning time: 0.396 ms
>  Execution time: 0.139 ms
> (9 rows)
> 
> Can not detach / attach / drop partition table.
> 
> Best regards,
> young
> 
> 
> yonj1e.github.io
> yangjie@highgo.com


-- 
Yugo Nagata <nagata@sraoss.co.jp>



Re: [HACKERS] [POC] hash partitioning

From
yangjie
Date:
font{ line-height: 1.7; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }
Hi,

This is my patch, before I forgot to add attachments, and the following address is also discussed.
https://www.postgresql.org/message-id/2017082612390093777512%40highgo.com
a#ntes-pcmail-signature-default:hover { text-decoration: underline; color: #199cff; cursor: pointer; } a#ntes-pcmail-signature-default:active { text-decoration: underline; color: #246fce; cursor: pointer; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }
font{ line-height: 1.7; }

-------

young

HighGo Database: http://www.highgo.com


On 8/28/2017 16:28Yugo Nagata<nagata@sraoss.co.jp> wrote:
Hi young,

On Mon, 28 Aug 2017 15:33:46 +0800
"yangjie@highgo.com" <yangjie@highgo.com> wrote:

> Hello

> Looking at your hash partitioning syntax, I implemented a hash partition in a more concise way, with no need to determine the number of sub-tables, and dynamically add partitions.

I think it is great work, but the current consensus about hash-partitioning supports 
Amul's patch[1], in which the syntax is different from the my original proposal. 
So, you will have to read Amul's patch and make a discussion if you still want to
propose your implementation.

Regards,

[1] https://www.postgresql.org/message-id/CAAJ_b965A2oog=6eFUhELexL3RmgFssB3G7LwkVA1bw0WUJJoA@mail.gmail.com



> Description

> The hash partition's implement is on the basis of the original range / list partition,and using similar syntax.

> To create a partitioned table ,use:

> CREATE TABLE h (id int) PARTITION BY HASH(id);

> The partitioning key supports only one value, and I think the partition key can support multiple values, 
> which may be difficult to implement when querying, but it is not impossible.

> A partition table can be create as bellow:

>  CREATE TABLE h1 PARTITION OF h;
>  CREATE TABLE h2 PARTITION OF h;
>  CREATE TABLE h3 PARTITION OF h;
>  
> FOR VALUES clause cannot be used, and the partition bound is calclulated automatically as partition index of single integer value.

> An inserted record is stored in a partition whose index equals 
> DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, values[0])) % nparts/* Number of partitions */
> ;
> In the above example, this is DatumGetUInt32(OidFunctionCall1(lookup_type_cache(key->parttypid[0], TYPECACHE_HASH_PROC)->hash_proc, id)) % 3;

> postgres=# insert into h select generate_series(1,20);
> INSERT 0 20
> postgres=# select tableoid::regclass,* from h;
>  tableoid | id 
> ----------+----
>  h1       |  3
>  h1       |  5
>  h1       | 17
>  h1       | 19
>  h2       |  2
>  h2       |  6
>  h2       |  7
>  h2       | 11
>  h2       | 12
>  h2       | 14
>  h2       | 15
>  h2       | 18
>  h2       | 20
>  h3       |  1
>  h3       |  4
>  h3       |  8
>  h3       |  9
>  h3       | 10
>  h3       | 13
>  h3       | 16
> (20 rows)

> The number of partitions here can be dynamically added, and if a new partition is created, the number of partitions changes, the calculated target partitions will change, and the same data is not reasonable in different partitions,So you need to re-calculate the existing data and insert the target partition when you create a new partition.

> postgres=# create table h4 partition of h;
> CREATE TABLE
> postgres=# select tableoid::regclass,* from h;
>  tableoid | id 
> ----------+----
>  h1       |  5
>  h1       | 17
>  h1       | 19
>  h1       |  6
>  h1       | 12
>  h1       |  8
>  h1       | 13
>  h2       | 11
>  h2       | 14
>  h3       |  1
>  h3       |  9
>  h3       |  2
>  h3       | 15
>  h4       |  3
>  h4       |  7
>  h4       | 18
>  h4       | 20
>  h4       |  4
>  h4       | 10
>  h4       | 16
> (20 rows)

> When querying the data, the hash partition uses the same algorithm as the insertion, and filters out the table that does not need to be scanned.

> postgres=# explain analyze select * from h where id = 1;
>                                              QUERY PLAN                                             
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..41.88 rows=13 width=4) (actual time=0.020..0.023 rows=1 loops=1)
>    ->  Seq Scan on h3  (cost=0.00..41.88 rows=13 width=4) (actual time=0.013..0.016 rows=1 loops=1)
>          Filter: (id = 1)
>          Rows Removed by Filter: 3
>  Planning time: 0.346 ms
>  Execution time: 0.061 ms
> (6 rows)

> postgres=# explain analyze select * from h where id in (1,5);;
>                                              QUERY PLAN                                             
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..83.75 rows=52 width=4) (actual time=0.016..0.028 rows=2 loops=1)
>    ->  Seq Scan on h1  (cost=0.00..41.88 rows=26 width=4) (actual time=0.015..0.018 rows=1 loops=1)
>          Filter: (id = ANY ('{1,5}'::integer[]))
>          Rows Removed by Filter: 6
>    ->  Seq Scan on h3  (cost=0.00..41.88 rows=26 width=4) (actual time=0.005..0.007 rows=1 loops=1)
>          Filter: (id = ANY ('{1,5}'::integer[]))
>          Rows Removed by Filter: 3
>  Planning time: 0.720 ms
>  Execution time: 0.074 ms
> (9 rows)

> postgres=# explain analyze select * from h where id = 1 or id = 5;;
>                                              QUERY PLAN                                             
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..96.50 rows=50 width=4) (actual time=0.017..0.078 rows=2 loops=1)
>    ->  Seq Scan on h1  (cost=0.00..48.25 rows=25 width=4) (actual time=0.015..0.019 rows=1 loops=1)
>          Filter: ((id = 1) OR (id = 5))
>          Rows Removed by Filter: 6
>    ->  Seq Scan on h3  (cost=0.00..48.25 rows=25 width=4) (actual time=0.005..0.010 rows=1 loops=1)
>          Filter: ((id = 1) OR (id = 5))
>          Rows Removed by Filter: 3
>  Planning time: 0.396 ms
>  Execution time: 0.139 ms
> (9 rows)

> Can not detach / attach / drop partition table.

> Best regards,
> young


> yonj1e.github.io
> yangjie@highgo.com


-- 
Yugo Nagata <nagata@sraoss.co.jp>
Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
I've updated patch to use an extended hash function (​Commit # 81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.

Regards,
Amul


On Thu, Jul 27, 2017 at 5:11 PM, amul sul <sulamul@gmail.com> wrote:
Attaching newer patches rebased against the latest master head. Thanks !

Regards,
Amul

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Rajkumar Raghuwanshi
Date:

On Mon, Sep 4, 2017 at 4:08 PM, amul sul <sulamul@gmail.com> wrote:
I've updated patch to use an extended hash function (​Commit # 81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.

I have done some testing with these patches, everything looks fine, attaching sql and out file for reference.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation
 
Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Mon, Sep 4, 2017 at 6:38 AM, amul sul <sulamul@gmail.com> wrote:
> I've updated patch to use an extended hash function (Commit #
> 81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.

Committed 0001 after noticing that Jeevan Ladhe also found that change
convenient for default partitioning.  I made a few minor cleanups;
hopefully I didn't break anything.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Fri, Sep 8, 2017 at 6:45 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Sep 4, 2017 at 6:38 AM, amul sul <sulamul@gmail.com> wrote:
> I've updated patch to use an extended hash function (Commit #
> 81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.

Committed 0001 after noticing that Jeevan Ladhe also found that change
convenient for default partitioning.  I made a few minor cleanups;
hopefully I didn't break anything.

​Thanks you.

Rebased 0002 against this commit & renamed to 0001, PFA. 

Regards,
Amul​
Attachment

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Fri, Sep 8, 2017 at 6:10 PM, amul sul <sulamul@gmail.com> wrote:
> On Fri, Sep 8, 2017 at 6:45 AM, Robert Haas <robertmhaas@gmail.com> wrote:
>>
>> On Mon, Sep 4, 2017 at 6:38 AM, amul sul <sulamul@gmail.com> wrote:
>> > I've updated patch to use an extended hash function (Commit #
>> > 81c5e46c490e2426db243eada186995da5bb0ba7) for the partitioning.
>>
>> Committed 0001 after noticing that Jeevan Ladhe also found that change
>> convenient for default partitioning.  I made a few minor cleanups;
>> hopefully I didn't break anything.
>
>
> Thanks you.
>
> Rebased 0002 against this commit & renamed to 0001, PFA.

Given that we have default partition support now, I am wondering
whether hash partitioned tables also should have default partitions.
The way we have structured hash partitioning syntax, there can be
"holes" in partitions. Default partition would help plug those holes.

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
>> Rebased 0002 against this commit & renamed to 0001, PFA.
>
> Given that we have default partition support now, I am wondering
> whether hash partitioned tables also should have default partitions.
> The way we have structured hash partitioning syntax, there can be
> "holes" in partitions. Default partition would help plug those holes.

Yeah, I was thinking about that, too.  On the one hand, it seems like
it's solving the problem the wrong way: if you've set up hash
partitioning properly, you shouldn't have any holes.  On the other
hand, supporting it probably wouldn't cost anything noticeable and
might make things seem more consistent.  I'm not sure which way to
jump on this one.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Alvaro Herrera
Date:
Robert Haas wrote:
> On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
> <ashutosh.bapat@enterprisedb.com> wrote:
> >> Rebased 0002 against this commit & renamed to 0001, PFA.
> >
> > Given that we have default partition support now, I am wondering
> > whether hash partitioned tables also should have default partitions.
> > The way we have structured hash partitioning syntax, there can be
> > "holes" in partitions. Default partition would help plug those holes.
> 
> Yeah, I was thinking about that, too.  On the one hand, it seems like
> it's solving the problem the wrong way: if you've set up hash
> partitioning properly, you shouldn't have any holes.  On the other
> hand, supporting it probably wouldn't cost anything noticeable and
> might make things seem more consistent.  I'm not sure which way to
> jump on this one.

How difficult/tedious/troublesome would be to install the missing
partitions if you set hash partitioning with a default partition and
only later on notice that some partitions are missing?  I think if the
answer is that you need to exclusive-lock something for a long time and
this causes a disruption in production systems, then it's better not to
allow a default partition at all and just force all the hash partitions
to be there from the start.

On the other hand, if you can get tuples out of the default partition
into their intended regular partitions without causing any disruption,
then it seems okay to allow default partitions in hash partitioning
setups.

(I, like many others, was unable to follow the default partition stuff
as closely as I would have liked.)

-- 
Álvaro Herrera                https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Mon, Sep 11, 2017 at 8:00 AM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:
> How difficult/tedious/troublesome would be to install the missing
> partitions if you set hash partitioning with a default partition and
> only later on notice that some partitions are missing?  I think if the
> answer is that you need to exclusive-lock something for a long time and
> this causes a disruption in production systems, then it's better not to
> allow a default partition at all and just force all the hash partitions
> to be there from the start.
>
> On the other hand, if you can get tuples out of the default partition
> into their intended regular partitions without causing any disruption,
> then it seems okay to allow default partitions in hash partitioning
> setups.

I think there's no real use case for default partitioning, and yeah,
you do need exclusive locks to repartition things (whether hash
partitioning or otherwise).  It would be nice to fix that eventually,
but it's hard, because the executor has to cope with the floor moving
under it, and as of today, it really can't cope with that at all - not
because of partitioning specifically, but because of existing design
decisions that will require a lot of work (and probably arguing) to
revisit.

I think the way to get around the usability issues for hash
partitioning is to eventually add some syntax that does things like
(1) automatically create the table with N properly-configured
partitions, (2) automatically split an existing partition into N
pieces, and (3) automatically rewrite the whole table using a
different partition count.

People seem to find the hash partitioning stuff a little arcane.  I
don't want to discount that confusion with some sort of high-handed "I
know better" attitude, I think the interface that users will actually
see can end up being pretty straightforward.  The complexity that is
there in the syntax is to allow pg_upgrade and pg_dump/restore to work
properly.  But users don't necessarily have to use the same syntax
that pg_dump does, just as you can say CREATE INDEX ON a (b) and let
the system specify the index name, but at dump time the index name is
specified explicitly.

> (I, like many others, was unable to follow the default partition stuff
> as closely as I would have liked.)

Uh, sorry about that.  Would it help if I wrote a blog post on it or
something?  The general idea is simple: any tuples that don't route to
any other partition get routed to the default partition.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Mon, Sep 11, 2017 at 5:30 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:
Robert Haas wrote:
> On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
> <ashutosh.bapat@enterprisedb.com> wrote:
> >> Rebased 0002 against this commit & renamed to 0001, PFA.
> >
> > Given that we have default partition support now, I am wondering
> > whether hash partitioned tables also should have default partitions.
> > The way we have structured hash partitioning syntax, there can be
> > "holes" in partitions. Default partition would help plug those holes.
>
> Yeah, I was thinking about that, too.  On the one hand, it seems like
> it's solving the problem the wrong way: if you've set up hash
> partitioning properly, you shouldn't have any holes.  On the other
> hand, supporting it probably wouldn't cost anything noticeable and
> might make things seem more consistent.  I'm not sure which way to
> jump on this one.

How difficult/tedious/troublesome would be to install the missing
partitions if you set hash partitioning with a default partition and
only later on notice that some partitions are missing?  I think if the
answer is that you need to exclusive-lock something for a long time and
this causes a disruption in production systems, then it's better not to
allow a default partition at all and just force all the hash partitions
to be there from the start.


I am also leaning toward ​not to support a default partition for a hash partitioned table.

The major drawback I can see is the constraint get created on the default partition
table.  IIUC, constraint on the default partition table are just negation of partition
constraint on all its sibling partitions.

Consider a hash partitioned table having partitions with (modulus 64, remainder 0) ,
...., (modulus 64, remainder 62) hash bound and partition column are col1, col2,...,so on,
then constraint for the default partition will be :

NOT( (satisfies_hash_partition(64, 0, hash_fn1(col1), hash_fn2(col2), ...) && ... &&
      satisfies_hash_partition(64, 62, hash_fn1(col1),hash_fn2(col2), ...))

​Which will be much harmful to the performance than any other partitioning
strategy because it calculate a hash for the same partitioning key multiple time.
We could overcome this by having an another SQL function (e.g satisfies_default_hash_partition)
which calculates hash value once and checks the remainder, and that would be
a different path from the current default partition framework.

​Regards,
Amul​

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
Hi Amul,

On 09/08/2017 08:40 AM, amul sul wrote:
> Rebased 0002 against this commit & renamed to 0001, PFA.
> 

This patch needs a rebase.

Best regards, Jesper



-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:
Hi Amul,

On 09/08/2017 08:40 AM, amul sul wrote:
Rebased 0002 against this commit & renamed to 0001, PFA.


This patch needs a rebase.


Thanks for your note.
​ ​
Attached is the patch rebased on the latest master head.
Also added error on
​creating ​
​d
efault partition
​for the hash partitioned table​
,
and updated document &
​ ​
test script for the same.

​Regards,
Amul​
 
Attachment

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
Hi Amul,

On 09/14/2017 04:58 AM, amul sul wrote:
> On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen <jesper.pedersen@redhat.com
>> This patch needs a rebase.
>>
>>
> Thanks for your note.
> ​ ​
> Attached is the patch rebased on the latest master head.
> Also added error on ​creating ​​default partition ​for the hash partitioned table​,
> and updated document & test script for the same.
> 

Thanks !

When I do

CREATE TABLE mytab (  a integer NOT NULL,  b integer NOT NULL,  c integer,  d integer
) PARTITION BY HASH (b);

and create 64 partitions;

CREATE TABLE mytab_p00 PARTITION OF mytab FOR VALUES WITH (MODULUS 64, 
REMAINDER 0);
...
CREATE TABLE mytab_p63 PARTITION OF mytab FOR VALUES WITH (MODULUS 64, 
REMAINDER 63);

and associated indexes

CREATE INDEX idx_p00 ON mytab_p00 USING btree (b, a);
...
CREATE INDEX idx_p63 ON mytab_p63 USING btree (b, a);

Populate the database, and do ANALYZE.

Given

EXPLAIN (ANALYZE, VERBOSE, BUFFERS ON) SELECT a, b, c, d FROM mytab 
WHERE b = 42

gives

Append  -> Index Scan using idx_p00 (cost rows=7) (actual rows=0)  ...  -> Index Scan using idx_p63 (cost rows=7)
(actualrows=0)
 

E.g. all partitions are being scanned. Of course one partition will 
contain the rows I'm looking for.

Best regards, Jesper


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Sep 14, 2017 at 11:39 AM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
> When I do
>
> CREATE TABLE mytab (
>   a integer NOT NULL,
>   b integer NOT NULL,
>   c integer,
>   d integer
> ) PARTITION BY HASH (b);
>
> and create 64 partitions;
>
> CREATE TABLE mytab_p00 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
> REMAINDER 0);
> ...
> CREATE TABLE mytab_p63 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
> REMAINDER 63);
>
> and associated indexes
>
> CREATE INDEX idx_p00 ON mytab_p00 USING btree (b, a);
> ...
> CREATE INDEX idx_p63 ON mytab_p63 USING btree (b, a);
>
> Populate the database, and do ANALYZE.
>
> Given
>
> EXPLAIN (ANALYZE, VERBOSE, BUFFERS ON) SELECT a, b, c, d FROM mytab WHERE b
> = 42
>
> gives
>
> Append
>   -> Index Scan using idx_p00 (cost rows=7) (actual rows=0)
>   ...
>   -> Index Scan using idx_p63 (cost rows=7) (actual rows=0)
>
> E.g. all partitions are being scanned. Of course one partition will contain
> the rows I'm looking for.

Yeah, we need Amit Langote's work in
http://postgr.es/m/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp
to land and this patch to be adapted to make use of it.  I think
that's the major thing still standing in the way of this. Concerns
were also raised about not having a way to see the hash function, but
we fixed that in 81c5e46c490e2426db243eada186995da5bb0ba7 and
hopefully this patch has been updated to use a seed (I haven't looked
yet).  And there was a concern about hash functions not being
portable, but the conclusion of that was basically that most people
think --load-via-partition-root will be a satisfactory workaround for
cases where that becomes a problem (cf. commit
23d7680d04b958de327be96ffdde8f024140d50e).  So this is the major
remaining issue that I know about.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
Hi,

On 09/14/2017 12:05 PM, Robert Haas wrote:
> On Thu, Sep 14, 2017 at 11:39 AM, Jesper Pedersen
> <jesper.pedersen@redhat.com> wrote:
>> When I do
>>
>> CREATE TABLE mytab (
>>    a integer NOT NULL,
>>    b integer NOT NULL,
>>    c integer,
>>    d integer
>> ) PARTITION BY HASH (b);
>>
>> and create 64 partitions;
>>
>> CREATE TABLE mytab_p00 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
>> REMAINDER 0);
>> ...
>> CREATE TABLE mytab_p63 PARTITION OF mytab FOR VALUES WITH (MODULUS 64,
>> REMAINDER 63);
>>
>> and associated indexes
>>
>> CREATE INDEX idx_p00 ON mytab_p00 USING btree (b, a);
>> ...
>> CREATE INDEX idx_p63 ON mytab_p63 USING btree (b, a);
>>
>> Populate the database, and do ANALYZE.
>>
>> Given
>>
>> EXPLAIN (ANALYZE, VERBOSE, BUFFERS ON) SELECT a, b, c, d FROM mytab WHERE b
>> = 42
>>
>> gives
>>
>> Append
>>    -> Index Scan using idx_p00 (cost rows=7) (actual rows=0)
>>    ...
>>    -> Index Scan using idx_p63 (cost rows=7) (actual rows=0)
>>
>> E.g. all partitions are being scanned. Of course one partition will contain
>> the rows I'm looking for.
> 
> Yeah, we need Amit Langote's work in
> http://postgr.es/m/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp
> to land and this patch to be adapted to make use of it.  I think
> that's the major thing still standing in the way of this. Concerns
> were also raised about not having a way to see the hash function, but
> we fixed that in 81c5e46c490e2426db243eada186995da5bb0ba7 and
> hopefully this patch has been updated to use a seed (I haven't looked
> yet).  And there was a concern about hash functions not being
> portable, but the conclusion of that was basically that most people
> think --load-via-partition-root will be a satisfactory workaround for
> cases where that becomes a problem (cf. commit
> 23d7680d04b958de327be96ffdde8f024140d50e).  So this is the major
> remaining issue that I know about.
> 

Thanks for the information, Robert !

Best regards, Jesper


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
David Fetter
Date:
On Mon, Sep 11, 2017 at 07:43:29AM -0400, Robert Haas wrote:
> On Mon, Sep 11, 2017 at 4:17 AM, Ashutosh Bapat
> <ashutosh.bapat@enterprisedb.com> wrote:
> >> Rebased 0002 against this commit & renamed to 0001, PFA.
> >
> > Given that we have default partition support now, I am wondering
> > whether hash partitioned tables also should have default
> > partitions.  The way we have structured hash partitioning syntax,
> > there can be "holes" in partitions. Default partition would help
> > plug those holes.
> 
> Yeah, I was thinking about that, too.  On the one hand, it seems
> like it's solving the problem the wrong way: if you've set up hash
> partitioning properly, you shouldn't have any holes.

Should we be pointing the gun away from people's feet by making hash
partitions that cover the space automagically when the partitioning
scheme[1] is specified?  In other words, do we have a good reason to have
only some of the hash partitions so defined by default?

Best,
David.

[1] For now, that's just the modulus, but the PoC included specifying
hashing functions, so I assume other ways to specify the partitioning
scheme could eventually be proposed.
-- 
David Fetter <david(at)fetter(dot)org> http://fetter.org/
Phone: +1 415 235 3778  AIM: dfetter666  Yahoo!: dfetter
Skype: davidfetter      XMPP: david(dot)fetter(at)gmail(dot)com

Remember to vote!
Consider donating to Postgres: http://www.postgresql.org/about/donate


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Sep 14, 2017 at 12:54 PM, David Fetter <david@fetter.org> wrote:
> Should we be pointing the gun away from people's feet by making hash
> partitions that cover the space automagically when the partitioning
> scheme[1] is specified?  In other words, do we have a good reason to have
> only some of the hash partitions so defined by default?

Sure, we can add some convenience syntax for that, but I'd like to get
the basic stuff working before doing that kind of polishing.

If nothing else, I assume Keith Fiske's pg_partman will provide a way
to magically DTRT about an hour after this goes in.  But probably we
can do better in core easily enough.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
On 09/14/2017 12:56 PM, Robert Haas wrote:
> On Thu, Sep 14, 2017 at 12:54 PM, David Fetter <david@fetter.org> wrote:
>> Should we be pointing the gun away from people's feet by making hash
>> partitions that cover the space automagically when the partitioning
>> scheme[1] is specified?  In other words, do we have a good reason to have
>> only some of the hash partitions so defined by default?
> 
> Sure, we can add some convenience syntax for that, but I'd like to get
> the basic stuff working before doing that kind of polishing.
> 
> If nothing else, I assume Keith Fiske's pg_partman will provide a way
> to magically DTRT about an hour after this goes in.  But probably we
> can do better in core easily enough.
> 

Yeah, it would be nice to have a syntax like

) PARTITION BY HASH (col) WITH (AUTO_CREATE = 64);

But then there also needs to be a way to create the 64 associated 
indexes too for everything to be easy.

Best regards, Jesper


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Sep 14, 2017 at 1:07 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
> Yeah, it would be nice to have a syntax like
>
> ) PARTITION BY HASH (col) WITH (AUTO_CREATE = 64);
>
> But then there also needs to be a way to create the 64 associated indexes
> too for everything to be easy.

Well, for that, there's this proposal:

http://postgr.es/m/c8fe4f6b-ff46-aae0-89e3-e936a35f0cfd@postgrespro.ru

As several people have right pointed out, there's a lot of work to be
done on partitioning it to get it to where we want it to be.  Even in
v10, it's got significant benefits, such as much faster bulk-loading,
but I don't hear anybody disputing the notion that a lot more work is
needed.  The good news is that a lot of that work is already in
progress; the bad news is that a lot of that work is not done yet.

But I think that's OK.  We can't solve every problem at once, and I
think we're moving things along here at a reasonably brisk pace.  That
didn't stop me from complaining bitterly to someone just yesterday
that we aren't moving faster still, but unfortunately EnterpriseDB has
only been able to get 12 developers to do any work at all on
partitioning this release cycle, and 3 of those have so far helped
only with review and benchmarking.  It's a pity we can't do more, but
considering how many community projects are 1-person efforts I think
it's pretty good.

To be clear, I know you're not (or at least I assume you're not)
trying to beat me up about this, just raising a concern, and I'm not
trying to beat you up either, just let you know that it is definitely
on the radar screen but not there yet.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
On 09/14/2017 01:52 PM, Robert Haas wrote:
> On Thu, Sep 14, 2017 at 1:07 PM, Jesper Pedersen
> <jesper.pedersen@redhat.com> wrote:
>> Yeah, it would be nice to have a syntax like
>>
>> ) PARTITION BY HASH (col) WITH (AUTO_CREATE = 64);
>>
>> But then there also needs to be a way to create the 64 associated indexes
>> too for everything to be easy.
> 
> Well, for that, there's this proposal:
> 
> http://postgr.es/m/c8fe4f6b-ff46-aae0-89e3-e936a35f0cfd@postgrespro.ru
> 
> As several people have right pointed out, there's a lot of work to be
> done on partitioning it to get it to where we want it to be.  Even in
> v10, it's got significant benefits, such as much faster bulk-loading,
> but I don't hear anybody disputing the notion that a lot more work is
> needed.  The good news is that a lot of that work is already in
> progress; the bad news is that a lot of that work is not done yet.
> 
> But I think that's OK.  We can't solve every problem at once, and I
> think we're moving things along here at a reasonably brisk pace.  That
> didn't stop me from complaining bitterly to someone just yesterday
> that we aren't moving faster still, but unfortunately EnterpriseDB has
> only been able to get 12 developers to do any work at all on
> partitioning this release cycle, and 3 of those have so far helped
> only with review and benchmarking.  It's a pity we can't do more, but
> considering how many community projects are 1-person efforts I think
> it's pretty good.
> 
> To be clear, I know you're not (or at least I assume you're not)
> trying to beat me up about this, just raising a concern, and I'm not
> trying to beat you up either, just let you know that it is definitely
> on the radar screen but not there yet.
> 

Definitely not a complain about the work being done.

I think the scope of Amul's and others work on hash partition support is 
where it needs to be. Improvements can always follow in future release.

My point was that is easy to script the definition of the partitions and 
their associated indexes, so it is more important to focus on the core 
functionality with the developer / review resources available.

However, it is a little bit difficult to follow the dependencies between 
different partition patches, so I may not always provide sane feedback, 
as seen in [1].

[1] 
https://www.postgresql.org/message-id/579077fd-8f07-aff7-39bc-b92c855cdb70%40redhat.com

Best regards, Jesper


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Sep 14, 2017 at 2:05 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
> However, it is a little bit difficult to follow the dependencies between
> different partition patches, so I may not always provide sane feedback, as
> seen in [1].
>
> [1]
> https://www.postgresql.org/message-id/579077fd-8f07-aff7-39bc-b92c855cdb70%40redhat.com

Yeah, no issues.  I knew about the dependency between those patches,
but I'm pretty sure there wasn't any terribly explicit discussion
about it, even if the issue probably came up parenthetically someplace
or other.  Oops.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Thom Brown
Date:
On 14 September 2017 at 09:58, amul sul <sulamul@gmail.com> wrote:
> On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen
> <jesper.pedersen@redhat.com> wrote:
>>
>> Hi Amul,
>>
>> On 09/08/2017 08:40 AM, amul sul wrote:
>>>
>>> Rebased 0002 against this commit & renamed to 0001, PFA.
>>>
>>
>> This patch needs a rebase.
>>
>
> Thanks for your note.
> Attached is the patch rebased on the latest master head.
> Also added error on
> creating
> d
> efault partition
> for the hash partitioned table
> ,
> and updated document &
> test script for the same.

Sorry, but this needs another rebase as it's broken by commit
77b6b5e9ceca04dbd6f0f6cd3fc881519acc8714.

Thom


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:

On Fri, Sep 15, 2017 at 4:30 AM, Thom Brown <thom@linux.com> wrote:
On 14 September 2017 at 09:58, amul sul <sulamul@gmail.com> wrote:
> On Wed, Sep 13, 2017 at 7:43 PM, Jesper Pedersen
> <jesper.pedersen@redhat.com> wrote:
>>
>> Hi Amul,
>>
>> On 09/08/2017 08:40 AM, amul sul wrote:
>>>
>>> Rebased 0002 against this commit & renamed to 0001, PFA.
>>>
>>
>> This patch needs a rebase.
>>
>
> Thanks for your note.
> Attached is the patch rebased on the latest master head.
> Also added error on
> creating
> d
> efault partition
> for the hash partitioned table
> ,
> and updated document &
> test script for the same.

Sorry, but this needs another rebase as it's broken by commit
77b6b5e9ceca04dbd6f0f6cd3fc881519acc8714.

Attached rebased patch, thanks.

Regards,
Amul

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
On 09/15/2017 02:30 AM, amul sul wrote:
> Attached rebased patch, thanks.
> 

While reading through the patch I thought it would be better to keep 
MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order 
to highlight that these are "keywords" for hash partition.

Also updated some of the documentation.

V20 patch passes make check-world, and my testing (typical 64 
partitions, and various ATTACH/DETACH scenarios).

Thanks for working on this !

Best regards,
  Jesper

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Mon, Sep 18, 2017 at 8:55 PM, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:
On 09/15/2017 02:30 AM, amul sul wrote:
Attached rebased patch, thanks.


While reading through the patch I thought it would be better to keep MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to highlight that these are "keywords" for hash partition.

Also updated some of the documentation.


Thanks a lot for the patch, included in the attached version.​
 
V20 patch passes make check-world, and my testing (typical 64 partitions, and various ATTACH/DETACH scenarios).

Nice, ​thanks again.

Regards,
Amul
Attachment

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
On 09/27/2017 03:05 AM, amul sul wrote:
>>> Attached rebased patch, thanks.
>>>
>>>
>> While reading through the patch I thought it would be better to keep
>> MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
>> highlight that these are "keywords" for hash partition.
>>
>> Also updated some of the documentation.
>>
>>
> Thanks a lot for the patch, included in the attached version.​
> 

Thank you.

Based on [1] I have moved the patch to "Ready for Committer".

[1] 
https://www.postgresql.org/message-id/CA%2BTgmoYsw3pusDen4_A44c7od%2BbEAST0eYo%2BjODtyofR0W2soQ%40mail.gmail.com

Best regards, Jesper



-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
On 2017/09/27 22:41, Jesper Pedersen wrote:
> On 09/27/2017 03:05 AM, amul sul wrote:
>>>> Attached rebased patch, thanks.
>>>>
>>>>
>>> While reading through the patch I thought it would be better to keep
>>> MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
>>> highlight that these are "keywords" for hash partition.
>>>
>>> Also updated some of the documentation.
>>>
>>>
>> Thanks a lot for the patch, included in the attached version.​
>>
> 
> Thank you.
> 
> Based on [1] I have moved the patch to "Ready for Committer".

Thanks a lot Amul for working on this.  Like Jesper said, the patch looks
pretty good overall.  I was looking at the latest version with intent to
study certain things about hash partitioning the way patch implements it,
during which I noticed some things.

+      The modulus must be a positive integer, and the remainder must a

must be a

+      suppose you have a hash-partitioned table with 8 children, each of
which
+      has modulus 8, but find it necessary to increase the number of
partitions
+      to 16.

Might it be a good idea to say 8 "partitions" instead of "children" in the
first sentence?

+      each modulus-8 partition until none remain.  While this may still
involve
+      a large amount of data movement at each step, it is still better than
+      having to create a whole new table and move all the data at once.
+     </para>
+

I read the paragraph that ends with the above text and started wondering
if the example to redistribute data in hash partitions by detaching and
attaching with new modulus/remainder could be illustrated with an example?
Maybe in the examples section of the ALTER TABLE page?

+      Since hash operator class provide only equality, not ordering,
collation

Either "Since hash operator classes provide" or "Since hash operator class
provides"

Other than the above points, patch looks good.


By the way, I noticed a couple of things about hash partition constraints:

1. In get_qual_for_hash(), using
get_fn_expr_rettype(&key->partsupfunc[i]), which returns InvalidOid for
the lack of fn_expr being set to non-NULL value, causes funcrettype of the
FuncExpr being generated for hashing partition key columns to be set to
InvalidOid, which I think is wrong.  That is, the following if condition
in get_fn_expr_rettype() is always satisfied:

    if (!flinfo || !flinfo->fn_expr)
        return InvalidOid;

I think we could use get_func_rettype(&key->partsupfunc[i].fn_oid)
instead.  Attached patch
hash-v21-set-funcexpr-funcrettype-correctly.patch, which applies on top
v21 of your patch.

2. It seems that the reason constraint exclusion doesn't work with hash
partitions as implemented by the patch is that predtest.c:
operator_predicate_proof() returns false even without looking into the
hash partition constraint, which is of the following form:

satisfies_hash_partition(<mod>, <rem>, <key1-exthash>,..)

beccause the above constraint expression doesn't translate into a a binary
opclause (an OpExpr), which operator_predicate_proof() knows how to work
with.  So, false is returned at the beginning of that function by the
following code:

    if (!is_opclause(predicate))
        return false;

For example,

create table p (a int) partition by hash (a);
create table p0 partition of p for values with (modulus 4, remainder 0);
create table p1 partition of p for values with (modulus 4, remainder 1);
\d+ p0
<...>
Partition constraint: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))

-- both p0 and p1 scanned
explain select * from p where satisfies_hash_partition(4, 0,
hashint4extended(a, '8816678312871386367'::bigint));
                                             QUERY PLAN

----------------------------------------------------------------------------------------------------
 Append  (cost=0.00..96.50 rows=1700 width=4)
   ->  Seq Scan on p0  (cost=0.00..48.25 rows=850 width=4)
         Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
   ->  Seq Scan on p1  (cost=0.00..48.25 rows=850 width=4)
         Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
'8816678312871386367'::bigint))
(5 rows)

-- both p0 and p1 scanned
explain select * from p where satisfies_hash_partition(4, 1,
hashint4extended(a, '8816678312871386367'::bigint));
                                             QUERY PLAN

----------------------------------------------------------------------------------------------------
 Append  (cost=0.00..96.50 rows=1700 width=4)
   ->  Seq Scan on p0  (cost=0.00..48.25 rows=850 width=4)
         Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
'8816678312871386367'::bigint))
   ->  Seq Scan on p1  (cost=0.00..48.25 rows=850 width=4)
         Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
'8816678312871386367'::bigint))
(5 rows)


I looked into how satisfies_hash_partition() works and came up with an
idea that I think will make constraint exclusion work.  What if we emitted
the hash partition constraint in the following form instead:

hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
                   <mod>) = <rem>

With that form, constraint exclusion seems to work as illustrated below:

\d+ p0
<...>
Partition constraint:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)

-- note only p0 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0;
                     QUERY PLAN


--------------------------------------------------------------------------------------------------------------------------
 Append  (cost=0.00..61.00 rows=13 width=4)
   ->  Seq Scan on p0  (cost=0.00..61.00 rows=13 width=4)
         Filter:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 0)
(3 rows)

-- note only p1 is scanned
explain select * from p where
hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 1;
                                                        QUERY PLAN


--------------------------------------------------------------------------------------------------------------------------
 Append  (cost=0.00..61.00 rows=13 width=4)
   ->  Seq Scan on p1  (cost=0.00..61.00 rows=13 width=4)
         Filter:
(hash_partition_modulus(hash_partition_hash(hashint4extended(a,
'8816678312871386367'::bigint)), 4) = 1)
(3 rows)

I tried to implement that in the attached
hash-v21-hash-part-constraint.patch, which applies on top v21 of your
patch (actually on top of
hash-v21-set-funcexpr-funcrettype-correctly.patch, which I think should be
applied anyway as it fixes a bug of the original patch).

What do you think?  Eventually, the new partition-pruning method [1] will
make using constraint exclusion obsolete, but it might be a good idea to
have it working if we can.

Thanks,
Amit

[1] https://commitfest.postgresql.org/14/1272/

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Thu, Sep 28, 2017 at 11:24 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
> On 2017/09/27 22:41, Jesper Pedersen wrote:
>> On 09/27/2017 03:05 AM, amul sul wrote:
>>>>> Attached rebased patch, thanks.
>>>>>
>>>>>
>>>> While reading through the patch I thought it would be better to keep
>>>> MODULUS and REMAINDER in caps, if CREATE TABLE was in caps too in order to
>>>> highlight that these are "keywords" for hash partition.
>>>>
>>>> Also updated some of the documentation.
>>>>
>>>>
>>> Thanks a lot for the patch, included in the attached version.
>>>
>>
>> Thank you.
>>
>> Based on [1] I have moved the patch to "Ready for Committer".
>
> Thanks a lot Amul for working on this.  Like Jesper said, the patch looks
> pretty good overall.  I was looking at the latest version with intent to
> study certain things about hash partitioning the way patch implements it,
> during which I noticed some things.
>

Thanks Amit for looking at the patch.

> +      The modulus must be a positive integer, and the remainder must a
>
> must be a
>

Fixed in the attached version.

> +      suppose you have a hash-partitioned table with 8 children, each of
> which
> +      has modulus 8, but find it necessary to increase the number of
> partitions
> +      to 16.
>

Fixed in the attached version.

> Might it be a good idea to say 8 "partitions" instead of "children" in the
> first sentence?
>
> +      each modulus-8 partition until none remain.  While this may still
> involve
> +      a large amount of data movement at each step, it is still better than
> +      having to create a whole new table and move all the data at once.
> +     </para>
> +
>

Fixed in the attached version.

> I read the paragraph that ends with the above text and started wondering
> if the example to redistribute data in hash partitions by detaching and
> attaching with new modulus/remainder could be illustrated with an example?
> Maybe in the examples section of the ALTER TABLE page?
>

I think hint in the documentation is more than enough. There is N number of
ways of data redistribution, the document is not meant to explain all of those.

> +      Since hash operator class provide only equality, not ordering,
> collation
>
> Either "Since hash operator classes provide" or "Since hash operator class
> provides"
>

Fixed in the attached version.

> Other than the above points, patch looks good.
>
>
> By the way, I noticed a couple of things about hash partition constraints:
>
> 1. In get_qual_for_hash(), using
> get_fn_expr_rettype(&key->partsupfunc[i]), which returns InvalidOid for
> the lack of fn_expr being set to non-NULL value, causes funcrettype of the
> FuncExpr being generated for hashing partition key columns to be set to
> InvalidOid, which I think is wrong.  That is, the following if condition
> in get_fn_expr_rettype() is always satisfied:
>
>     if (!flinfo || !flinfo->fn_expr)
>         return InvalidOid;
>
> I think we could use get_func_rettype(&key->partsupfunc[i].fn_oid)
> instead.  Attached patch
> hash-v21-set-funcexpr-funcrettype-correctly.patch, which applies on top
> v21 of your patch.
>

Thanks for the patch, included in the attached version.

> 2. It seems that the reason constraint exclusion doesn't work with hash
> partitions as implemented by the patch is that predtest.c:
> operator_predicate_proof() returns false even without looking into the
> hash partition constraint, which is of the following form:
>
> satisfies_hash_partition(<mod>, <rem>, <key1-exthash>,..)
>
> beccause the above constraint expression doesn't translate into a a binary
> opclause (an OpExpr), which operator_predicate_proof() knows how to work
> with.  So, false is returned at the beginning of that function by the
> following code:
>
>     if (!is_opclause(predicate))
>         return false;
>
> For example,
>
> create table p (a int) partition by hash (a);
> create table p0 partition of p for values with (modulus 4, remainder 0);
> create table p1 partition of p for values with (modulus 4, remainder 1);
> \d+ p0
> <...>
> Partition constraint: satisfies_hash_partition(4, 0, hashint4extended(a,
> '8816678312871386367'::bigint))
>
> -- both p0 and p1 scanned
> explain select * from p where satisfies_hash_partition(4, 0,
> hashint4extended(a, '8816678312871386367'::bigint));
>                                              QUERY PLAN
>
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..96.50 rows=1700 width=4)
>    ->  Seq Scan on p0  (cost=0.00..48.25 rows=850 width=4)
>          Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
> '8816678312871386367'::bigint))
>    ->  Seq Scan on p1  (cost=0.00..48.25 rows=850 width=4)
>          Filter: satisfies_hash_partition(4, 0, hashint4extended(a,
> '8816678312871386367'::bigint))
> (5 rows)
>
> -- both p0 and p1 scanned
> explain select * from p where satisfies_hash_partition(4, 1,
> hashint4extended(a, '8816678312871386367'::bigint));
>                                              QUERY PLAN
>
> ----------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..96.50 rows=1700 width=4)
>    ->  Seq Scan on p0  (cost=0.00..48.25 rows=850 width=4)
>          Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
> '8816678312871386367'::bigint))
>    ->  Seq Scan on p1  (cost=0.00..48.25 rows=850 width=4)
>          Filter: satisfies_hash_partition(4, 1, hashint4extended(a,
> '8816678312871386367'::bigint))
> (5 rows)
>
>
> I looked into how satisfies_hash_partition() works and came up with an
> idea that I think will make constraint exclusion work.  What if we emitted
> the hash partition constraint in the following form instead:
>
> hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
>                    <mod>) = <rem>
>
> With that form, constraint exclusion seems to work as illustrated below:
>
> \d+ p0
> <...>
> Partition constraint:
> (hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 0)
>
> -- note only p0 is scanned
> explain select * from p where
> hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 0;
>                      QUERY PLAN
>
>
--------------------------------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..61.00 rows=13 width=4)
>    ->  Seq Scan on p0  (cost=0.00..61.00 rows=13 width=4)
>          Filter:
> (hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 0)
> (3 rows)
>
> -- note only p1 is scanned
> explain select * from p where
> hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 1;
>                                                         QUERY PLAN
>
>
--------------------------------------------------------------------------------------------------------------------------
>  Append  (cost=0.00..61.00 rows=13 width=4)
>    ->  Seq Scan on p1  (cost=0.00..61.00 rows=13 width=4)
>          Filter:
> (hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 1)
> (3 rows)
>
> I tried to implement that in the attached
> hash-v21-hash-part-constraint.patch, which applies on top v21 of your
> patch (actually on top of
> hash-v21-set-funcexpr-funcrettype-correctly.patch, which I think should be
> applied anyway as it fixes a bug of the original patch).
>
> What do you think?  Eventually, the new partition-pruning method [1] will
> make using constraint exclusion obsolete, but it might be a good idea to
> have it working if we can.
>

It does not really do the partition pruning via constraint exclusion and I don't
think anyone is going to use the remainder in the where condition to fetch
data and hash partitioning is not meant for that.

But I am sure that we could solve this problem using your and Beena's work
toward faster partition pruning[1] and Runtime Partition Pruning[2].

Will think on this changes if it is required for the pruning feature.

Regards,
Amul

1] https://postgr.es/m/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp
2] https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@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

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Sep 28, 2017 at 1:54 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
> I looked into how satisfies_hash_partition() works and came up with an
> idea that I think will make constraint exclusion work.  What if we emitted
> the hash partition constraint in the following form instead:
>
> hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
>                    <mod>) = <rem>
>
> With that form, constraint exclusion seems to work as illustrated below:
>
> \d+ p0
> <...>
> Partition constraint:
> (hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 0)
>
> -- note only p0 is scanned
> explain select * from p where
> hash_partition_modulus(hash_partition_hash(hashint4extended(a,
> '8816678312871386367'::bigint)), 4) = 0;

What we actually want constraint exclusion to cover is SELECT * FROM p
WHERE a = 525600;

As Amul says, nobody's going to enter a query in the form you have it
here.  Life is too short to take time to put queries into bizarre
forms.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Jesper Pedersen
Date:
Hi Amul,

On 09/28/2017 05:56 AM, amul sul wrote:
> It does not really do the partition pruning via constraint exclusion and I don't
> think anyone is going to use the remainder in the where condition to fetch
> data and hash partitioning is not meant for that.
> 
> But I am sure that we could solve this problem using your and Beena's work
> toward faster partition pruning[1] and Runtime Partition Pruning[2].
> 
> Will think on this changes if it is required for the pruning feature.
> 

Could you rebase on latest master ?

Best regards, Jesper


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Fri, Oct 6, 2017 at 5:35 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:
> Hi Amul,
>
> Could you rebase on latest master ?
>

Sure will post that soon, but before that, I need to test hash partitioning
with recent partition-wise join commit (f49842d1ee), thanks.

Regards,
Amul


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Sat, Oct 7, 2017 at 5:22 PM, amul sul <sulamul@gmail.com> wrote:
> On Fri, Oct 6, 2017 at 5:35 PM, Jesper Pedersen
> <jesper.pedersen@redhat.com> wrote:
>> Hi Amul,
>>
>> Could you rebase on latest master ?
>>
>
> Sure will post that soon, but before that, I need to test hash partitioning
> with recent partition-wise join commit (f49842d1ee), thanks.
>

Updated patch attached.

0001 is the rebased of the previous patch, no new change.
0002 few changes in partition-wise join code to support
hash-partitioned table as well & regression tests.

Thanks & Regards,
Amul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Mon, Oct 9, 2017 at 4:44 PM, amul sul <sulamul@gmail.com> wrote:


> 0002 few changes in partition-wise join code to support
> hash-partitioned table as well & regression tests.

+    switch (key->strategy)
+    {
+        case PARTITION_STRATEGY_HASH:
+            /*
+             * Indexes array is same as the greatest modulus.
+             * See partition_bounds_equal() for more explanation.
+             */
+            num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]);
+            break;
This logic is duplicated at multiple places.  I think it's time we consolidate
these changes in a function/macro and call it from the places where we have to
calculate number of indexes based on the information in partition descriptor.
Refactoring existing code might be a separate patch and then add hash
partitioning case in hash partitioning patch.

+        int        dim = hash_part? 2 : partnatts;
Call the variable as natts_per_datum or just natts?

+                                    hash_part? true : key->parttypbyval[j],
+                                    key->parttyplen[j]);
parttyplen is the length of partition key attribute, whereas what you want here
is the length of type of modulus and remainder. Is that correct? Probably we
need some special handling wherever parttyplen and parttypbyval is used e.g. in
call to partition_bounds_equal() from build_joinrel_partition_info().

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Mon, Oct 9, 2017 at 5:51 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
> On Mon, Oct 9, 2017 at 4:44 PM, amul sul <sulamul@gmail.com> wrote:
>

Thanks Ashutosh for your review, please find my comment inline.

>
>> 0002 few changes in partition-wise join code to support
>> hash-partitioned table as well & regression tests.
>
> +    switch (key->strategy)
> +    {
> +        case PARTITION_STRATEGY_HASH:
> +            /*
> +             * Indexes array is same as the greatest modulus.
> +             * See partition_bounds_equal() for more explanation.
> +             */
> +            num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]);
> +            break;
> This logic is duplicated at multiple places.  I think it's time we consolidate
> these changes in a function/macro and call it from the places where we have to
> calculate number of indexes based on the information in partition descriptor.
> Refactoring existing code might be a separate patch and then add hash
> partitioning case in hash partitioning patch.
>

Make sense, added get_partition_bound_num_indexes() to get number of index
elements in 0001 & get_greatest_modulus() as name suggested to get the greatest
modulus of the hash partition bound in 0002.

> +        int        dim = hash_part? 2 : partnatts;
> Call the variable as natts_per_datum or just natts?
>

natts represents the number of attributes, but for the hash partition bound we
are not dealing with the attribute so that I have used short-form of dimension,
thoughts?

> +                                    hash_part? true : key->parttypbyval[j],
> +                                    key->parttyplen[j]);
> parttyplen is the length of partition key attribute, whereas what you want here
> is the length of type of modulus and remainder. Is that correct? Probably we
> need some special handling wherever parttyplen and parttypbyval is used e.g. in
> call to partition_bounds_equal() from build_joinrel_partition_info().
>

Unless I am missing something, I don't think we should worry about parttyplen
because in the datumCopy() when the datatype is pass-by-value then typelen
is ignored.

Regards,
Amul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
> On Mon, Oct 9, 2017 at 5:51 PM, Ashutosh Bapat
> <ashutosh.bapat@enterprisedb.com> wrote:
>> On Mon, Oct 9, 2017 at 4:44 PM, amul sul <sulamul@gmail.com> wrote:
>>
>
> Thanks Ashutosh for your review, please find my comment inline.
>
>>
>>> 0002 few changes in partition-wise join code to support
>>> hash-partitioned table as well & regression tests.
>>
>> +    switch (key->strategy)
>> +    {
>> +        case PARTITION_STRATEGY_HASH:
>> +            /*
>> +             * Indexes array is same as the greatest modulus.
>> +             * See partition_bounds_equal() for more explanation.
>> +             */
>> +            num_indexes = DatumGetInt32(src->datums[ndatums - 1][0]);
>> +            break;
>> This logic is duplicated at multiple places.  I think it's time we consolidate
>> these changes in a function/macro and call it from the places where we have to
>> calculate number of indexes based on the information in partition descriptor.
>> Refactoring existing code might be a separate patch and then add hash
>> partitioning case in hash partitioning patch.
>>
>
> Make sense, added get_partition_bound_num_indexes() to get number of index
> elements in 0001 & get_greatest_modulus() as name suggested to get the greatest
> modulus of the hash partition bound in 0002.
>
>> +        int        dim = hash_part? 2 : partnatts;
>> Call the variable as natts_per_datum or just natts?
>>
>
> natts represents the number of attributes, but for the hash partition bound we
> are not dealing with the attribute so that I have used short-form of dimension,
> thoughts?

Okay, I think the dimension(dim) is also unfit here.  Any suggestions?

>
>> +                                    hash_part? true : key->parttypbyval[j],
>> +                                    key->parttyplen[j]);
>> parttyplen is the length of partition key attribute, whereas what you want here
>> is the length of type of modulus and remainder. Is that correct? Probably we
>> need some special handling wherever parttyplen and parttypbyval is used e.g. in
>> call to partition_bounds_equal() from build_joinrel_partition_info().
>>
>
> Unless I am missing something, I don't think we should worry about parttyplen
> because in the datumCopy() when the datatype is pass-by-value then typelen
> is ignored.
>
> Regards,
> Amul


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:

>> +                                    hash_part? true : key->parttypbyval[j],
>> +                                    key->parttyplen[j]);
>> parttyplen is the length of partition key attribute, whereas what you want here
>> is the length of type of modulus and remainder. Is that correct? Probably we
>> need some special handling wherever parttyplen and parttypbyval is used e.g. in
>> call to partition_bounds_equal() from build_joinrel_partition_info().
>>
>
> Unless I am missing something, I don't think we should worry about parttyplen
> because in the datumCopy() when the datatype is pass-by-value then typelen
> is ignored.

That's true, but it's ugly, passing typbyvalue of one type and len of other.

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Tue, Oct 10, 2017 at 3:40 PM, amul sul <sulamul@gmail.com> wrote:
>>
>> natts represents the number of attributes, but for the hash partition bound we
>> are not dealing with the attribute so that I have used short-form of dimension,
>> thoughts?
>
> Okay, I think the dimension(dim) is also unfit here.  Any suggestions?
>


I think natts is ok, since we are dealing with the number of
attributes in the pack of datums; esp. when ndatums is already taken.

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Tue, Oct 10, 2017 at 3:42 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
> On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
>
>>> +                                    hash_part? true : key->parttypbyval[j],
>>> +                                    key->parttyplen[j]);
>>> parttyplen is the length of partition key attribute, whereas what you want here
>>> is the length of type of modulus and remainder. Is that correct? Probably we
>>> need some special handling wherever parttyplen and parttypbyval is used e.g. in
>>> call to partition_bounds_equal() from build_joinrel_partition_info().
>>>
>>
>> Unless I am missing something, I don't think we should worry about parttyplen
>> because in the datumCopy() when the datatype is pass-by-value then typelen
>> is ignored.
>
> That's true, but it's ugly, passing typbyvalue of one type and len of other.
>

How about the attached patch(0003)?
Also, the dim variable is renamed to natts.

Regards,
Amul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Tue, Oct 10, 2017 at 7:07 AM, amul sul <sulamul@gmail.com> wrote:
> How about the attached patch(0003)?
> Also, the dim variable is renamed to natts.

I'm not sure I believe this comment:

+        /*
+         * We arrange the partitions in the ascending order of their modulus
+         * and remainders.  Also every modulus is factor of next larger
+         * modulus.  This means that the index of a given partition is same as
+         * the remainder of that partition.  Also entries at (remainder + N *
+         * modulus) positions in indexes array are all same for (modulus,
+         * remainder) specification for any partition.  Thus datums array from
+         * both the given bounds are same, if and only if their indexes array
+         * will be same.  So, it suffices to compare indexes array.
+         */

I am particularly not sure that I believe that the index of a
partition must be the same as the remainder.  It doesn't seem like
that would be true when there is more than one modulus or when some
partitions are missing.

+                    if (offset < 0)
+                    {
+                        next_modulus = DatumGetInt32(datums[0][0]);
+                        valid_modulus = (next_modulus % spec->modulus) == 0;
+                    }
+                    else
+                    {
+                        prev_modulus = DatumGetInt32(datums[offset][0]);
+                        valid_modulus = (spec->modulus % prev_modulus) == 0;
+
+                        if (valid_modulus && (offset + 1) < ndatums)
+                        {
+                            next_modulus =
DatumGetInt32(datums[offset + 1][0]);
+                            valid_modulus = (next_modulus %
spec->modulus) == 0;
+                        }
+                    }

I don't think this is quite right.  It checks the new modulus against
prev_modulus whenever prev_modulus is defined, which is correct, but
it doesn't check the new modulus against the next_modulus except when
offset < 0.  But actually that check needs to be performed, I think,
whenever the new modulus is less than the greatest modulus seen so
far.

+ * For a partitioned table defined as:
+ *    CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash
+ *    FOR VALUES WITH (MODULUS 2, REMAINDER 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash
+ *    FOR VALUES WITH (MODULUS 4, REMAINDER 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash
+ *    FOR VALUES WITH (MODULUS 8, REMAINDER 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash
+ *    FOR VALUES WITH (MODULUS 8, REMAINDER 4);
+ *
+ * This function will return one of the following in the form of an
+ * expression:
+ *
+ * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
+ *                                             hash_fn_2_extended(b,
HASH_SEED))
+ * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
+ *                                             hash_fn_2_extended(b,
HASH_SEED))
+ * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
+ *                                             hash_fn_2_extended(b,
HASH_SEED))
+ * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
+ *                                             hash_fn_2_extended(b,
HASH_SEED))

I think instead of this lengthy example you should try to explain the
general rule.  Maybe something like: the partition constraint for a
hash partition is always a call to the built-in function
satisfies_hash_partition().  The first two arguments are the modulus
and remainder for the partition; the remaining arguments are the hash
values computed for each column of the partition key using the
extended hash function from the appropriate opclass.

+static uint64
+mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)

It would be nice to use the hash_combine() facility Andres recently
added for this rather than having a way to do it that is specific to
hash partitioning, but that function only works for 32-bit hash
values.  Maybe we can persuade Andres to add a hash_combine64...

+         * a hash operator class

Missing period at end.

+        if (strategy == PARTITION_STRATEGY_HASH)
+            ereport(ERROR,
+                    (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+                     errmsg("default hash partition is not supported")));

Maybe errmsg("a hash-partitioned table may not have a default partition")?

+/* Seed for the extended hash function */
+#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)

I suggest HASH_PARTITION_SEED -- this is too generic.

Have you checked how well the tests you've added cover the code you've
added?  What code is not covered by the tests, and is there any way to
cover it?

Thanks,

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Amit Langote
Date:
On 2017/09/30 1:53, Robert Haas wrote:
> On Thu, Sep 28, 2017 at 1:54 AM, Amit Langote
> <Langote_Amit_f8@lab.ntt.co.jp> wrote:
>> I looked into how satisfies_hash_partition() works and came up with an
>> idea that I think will make constraint exclusion work.  What if we emitted
>> the hash partition constraint in the following form instead:
>>
>> hash_partition_mod(hash_partition_hash(key1-exthash, key2-exthash),
>>                    <mod>) = <rem>
>>
>> With that form, constraint exclusion seems to work as illustrated below:
>>
>> \d+ p0
>> <...>
>> Partition constraint:
>> (hash_partition_modulus(hash_partition_hash(hashint4extended(a,
>> '8816678312871386367'::bigint)), 4) = 0)
>>
>> -- note only p0 is scanned
>> explain select * from p where
>> hash_partition_modulus(hash_partition_hash(hashint4extended(a,
>> '8816678312871386367'::bigint)), 4) = 0;
> 
> What we actually want constraint exclusion to cover is SELECT * FROM p
> WHERE a = 525600;

I agree.

> As Amul says, nobody's going to enter a query in the form you have it
> here.  Life is too short to take time to put queries into bizarre
> forms.

Here too.  I was falsely thinking that satisfies_hash_partition() is
intended to be used for more than just enforcing the partition constraint
when data is directly inserted into a hash partition, or more precisely to
be used in the CHECK constraint of the table that is to be attached as a
hash partition.  Now, we ask users to add such a constraint to avoid the
constraint validation scan, because the system knows how to infer from the
constraint that the partition constraint is satisfied.  I observed however
that, unlike range and list partitioning, the hash partition's constraint
could only ever be implied because of structural equality (equal()'ness)
of the existing constraint expression and the partition constraint
expression.  For example, a more restrictive range or list qual implies
the partition constraint, but it requires performing btree operator based
proof.  The proof is impossible with the chosen structure of hash
partitioning constraint, but it seems that that's OK.  That is, it's OK to
ask users to add the exact constraint (matching modulus and reminder
values in the call to satisfies_hash_partition() specified in the CHECK
constraint) to avoid the validation scan.

Thanks,
Amit



-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Thu, Oct 12, 2017 at 6:31 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> On Tue, Oct 10, 2017 at 7:07 AM, amul sul <sulamul@gmail.com> wrote:
>> How about the attached patch(0003)?
>> Also, the dim variable is renamed to natts.
>
> I'm not sure I believe this comment:
>
> +        /*
> +         * We arrange the partitions in the ascending order of their modulus
> +         * and remainders.  Also every modulus is factor of next larger
> +         * modulus.  This means that the index of a given partition is same as
> +         * the remainder of that partition.  Also entries at (remainder + N *
> +         * modulus) positions in indexes array are all same for (modulus,
> +         * remainder) specification for any partition.  Thus datums array from
> +         * both the given bounds are same, if and only if their indexes array
> +         * will be same.  So, it suffices to compare indexes array.
> +         */
>
> I am particularly not sure that I believe that the index of a
> partition must be the same as the remainder.  It doesn't seem like
> that would be true when there is more than one modulus or when some
> partitions are missing.
>

Looks like an explanation by the comment is not good enough, will think on this.

Here are the links for the previous discussion:
1] https://postgr.es/m/CAFjFpRfHqSGBjNgJV2p%2BC4Yr5Qxvwygdsg4G_VQ6q9NTB-i3MA%40mail.gmail.com
2] https://postgr.es/m/CAFjFpRdeESKFkVGgmOdYvmD3d56-58c5VCBK0zDRjHrkq_VcNg%40mail.gmail.com


> +                    if (offset < 0)
> +                    {
> +                        next_modulus = DatumGetInt32(datums[0][0]);
> +                        valid_modulus = (next_modulus % spec->modulus) == 0;
> +                    }
> +                    else
> +                    {
> +                        prev_modulus = DatumGetInt32(datums[offset][0]);
> +                        valid_modulus = (spec->modulus % prev_modulus) == 0;
> +
> +                        if (valid_modulus && (offset + 1) < ndatums)
> +                        {
> +                            next_modulus =
> DatumGetInt32(datums[offset + 1][0]);
> +                            valid_modulus = (next_modulus %
> spec->modulus) == 0;
> +                        }
> +                    }
>
> I don't think this is quite right.  It checks the new modulus against
> prev_modulus whenever prev_modulus is defined, which is correct, but
> it doesn't check the new modulus against the next_modulus except when
> offset < 0.  But actually that check needs to be performed, I think,
> whenever the new modulus is less than the greatest modulus seen so
> far.
>
It does. See the "if (valid_modulus && (offset + 1) < ndatums)"  block in the
else part of the snippet that you are referring.

For e.g new modulus 25 & 150 is not accepted for the hash partitioned bound with
modulus 10,50,200. Will cover this test as well.

> + * For a partitioned table defined as:
> + *    CREATE TABLE simple_hash (a int, b char(10)) PARTITION BY HASH (a, b);
> + *
> + * CREATE TABLE p_p1 PARTITION OF simple_hash
> + *    FOR VALUES WITH (MODULUS 2, REMAINDER 1);
> + * CREATE TABLE p_p2 PARTITION OF simple_hash
> + *    FOR VALUES WITH (MODULUS 4, REMAINDER 2);
> + * CREATE TABLE p_p3 PARTITION OF simple_hash
> + *    FOR VALUES WITH (MODULUS 8, REMAINDER 0);
> + * CREATE TABLE p_p4 PARTITION OF simple_hash
> + *    FOR VALUES WITH (MODULUS 8, REMAINDER 4);
> + *
> + * This function will return one of the following in the form of an
> + * expression:
> + *
> + * for p_p1: satisfies_hash_partition(2, 1, hash_fn_1_extended(a, HASH_SEED),
> + *                                             hash_fn_2_extended(b,
> HASH_SEED))
> + * for p_p2: satisfies_hash_partition(4, 2, hash_fn_1_extended(a, HASH_SEED),
> + *                                             hash_fn_2_extended(b,
> HASH_SEED))
> + * for p_p3: satisfies_hash_partition(8, 0, hash_fn_1_extended(a, HASH_SEED),
> + *                                             hash_fn_2_extended(b,
> HASH_SEED))
> + * for p_p4: satisfies_hash_partition(8, 4, hash_fn_1_extended(a, HASH_SEED),
> + *                                             hash_fn_2_extended(b,
> HASH_SEED))
>
> I think instead of this lengthy example you should try to explain the
> general rule.  Maybe something like: the partition constraint for a
> hash partition is always a call to the built-in function
> satisfies_hash_partition().  The first two arguments are the modulus
> and remainder for the partition; the remaining arguments are the hash
> values computed for each column of the partition key using the
> extended hash function from the appropriate opclass.
>
Okay will add this.

> +static uint64
> +mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
>
How about combining high 32 bits and the low 32 bits separately as shown below?

static inline uint64
hash_combine64(uint64 a, uint64 b)
{   return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)           | hash_combine((unit32) a,
(unit32)b));
 
}

> It would be nice to use the hash_combine() facility Andres recently
> added for this rather than having a way to do it that is specific to
> hash partitioning, but that function only works for 32-bit hash
> values.  Maybe we can persuade Andres to add a hash_combine64...
>
> +         * a hash operator class
>
> Missing period at end.
>
Okay will fix this.

> +        if (strategy == PARTITION_STRATEGY_HASH)
> +            ereport(ERROR,
> +                    (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
> +                     errmsg("default hash partition is not supported")));
>
> Maybe errmsg("a hash-partitioned table may not have a default partition")?
>
Okay will add this.

> +/* Seed for the extended hash function */
> +#define HASH_SEED UINT64CONST(0x7A5B22367996DCFF)
>
> I suggest HASH_PARTITION_SEED -- this is too generic.
>
Okay will add this.

> Have you checked how well the tests you've added cover the code you've
> added?  What code is not covered by the tests, and is there any way to
> cover it?
>
Will try to get gcov report for this patch.

Thanks for your review.

Regards,
Amul


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Oct 12, 2017 at 9:08 AM, amul sul <sulamul@gmail.com> wrote:
> How about combining high 32 bits and the low 32 bits separately as shown below?
>
> static inline uint64
> hash_combine64(uint64 a, uint64 b)
> {
>     return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
>             | hash_combine((unit32) a, (unit32) b));
> }

I doubt that's the best approach, but I don't have something specific
to recommend.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Andres Freund
Date:
On 2017-10-12 10:05:26 -0400, Robert Haas wrote:
> On Thu, Oct 12, 2017 at 9:08 AM, amul sul <sulamul@gmail.com> wrote:
> > How about combining high 32 bits and the low 32 bits separately as shown below?
> >
> > static inline uint64
> > hash_combine64(uint64 a, uint64 b)
> > {
> >     return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
> >             | hash_combine((unit32) a, (unit32) b));
> > }
> 
> I doubt that's the best approach, but I don't have something specific
> to recommend.

Yea, that doesn't look great. There's basically no intermixing between
low and high 32 bits. going on.  We probably should just expand the
concept of the 32 bit function:

static inline uint32
hash_combine32(uint32 a, uint32 b)
{       /* 0x9e3779b9 is the golden ratio reciprocal */a ^= b + 0x9e3779b9 + (a << 6) + (a >> 2);return a;
}

to something roughly like:

static inline uint64
hash_combine64(uint64 a, uint64 b)
{       /* 0x49A0F4DD15E5A8E3 is 64bit random data */a ^= b + 0x49A0F4DD15E5A8E3 + (a << 54) + (a >> 7);return a;
}

In contrast to the 32 bit version's fancy use of the golden ratio
reciprocal as a constant I went brute force, and just used 64bit of
/dev/random. From my understanding the important property is that bits
are independent from each other, nothing else.

The shift widths are fairly random, but they should bring in enough bit
perturbation when mixing in only 32bit of hash value (i.e
0x00000000xxxxxxxx).

Are we going to rely on the the combine function to stay the same
forever after?

Greetings,

Andres Freund


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Oct 12, 2017 at 3:43 PM, Andres Freund <andres@anarazel.de> wrote:
> Are we going to rely on the the combine function to stay the same
> forever after?

If we change them, it will be a pg_upgrade compatibility break for
anyone using hash-partitioned tables with more than one partitioning
column.  Dump and reload will also break unless
--load-via-partition-root is used.

In other words, it's not utterly fixed in stone --- we invented
--load-via-partition-root primarily to cope with circumstances that
could change hash values --- but we sure don't want to be changing it
with any regularity, or for a less-than-excellent reason.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Andres Freund
Date:
On 2017-10-12 16:06:11 -0400, Robert Haas wrote:
> On Thu, Oct 12, 2017 at 3:43 PM, Andres Freund <andres@anarazel.de> wrote:
> > Are we going to rely on the the combine function to stay the same
> > forever after?
> 
> If we change them, it will be a pg_upgrade compatibility break for
> anyone using hash-partitioned tables with more than one partitioning
> column.  Dump and reload will also break unless
> --load-via-partition-root is used.
> 
> In other words, it's not utterly fixed in stone --- we invented
> --load-via-partition-root primarily to cope with circumstances that
> could change hash values --- but we sure don't want to be changing it
> with any regularity, or for a less-than-excellent reason.

Yea, that's what I expected. It'd probably good for somebody to run
smhasher or such on the output of the combine function (or even better,
on both the 32 and 64 bit variants) in that case.

Greetings,

Andres Freund


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Oct 12, 2017 at 4:20 PM, Andres Freund <andres@anarazel.de> wrote:
>> In other words, it's not utterly fixed in stone --- we invented
>> --load-via-partition-root primarily to cope with circumstances that
>> could change hash values --- but we sure don't want to be changing it
>> with any regularity, or for a less-than-excellent reason.
>
> Yea, that's what I expected. It'd probably good for somebody to run
> smhasher or such on the output of the combine function (or even better,
> on both the 32 and 64 bit variants) in that case.

Not sure how that test suite works exactly, but presumably the
characteristics in practice will depend the behavior of the hash
functions used as input the combine function - so the behavior could
be good for an (int, int) key but bad for a (text, date) key, or
whatever.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Andres Freund
Date:
On 2017-10-12 17:27:52 -0400, Robert Haas wrote:
> On Thu, Oct 12, 2017 at 4:20 PM, Andres Freund <andres@anarazel.de> wrote:
> >> In other words, it's not utterly fixed in stone --- we invented
> >> --load-via-partition-root primarily to cope with circumstances that
> >> could change hash values --- but we sure don't want to be changing it
> >> with any regularity, or for a less-than-excellent reason.
> >
> > Yea, that's what I expected. It'd probably good for somebody to run
> > smhasher or such on the output of the combine function (or even better,
> > on both the 32 and 64 bit variants) in that case.
> 
> Not sure how that test suite works exactly, but presumably the
> characteristics in practice will depend the behavior of the hash
> functions used as input the combine function - so the behavior could
> be good for an (int, int) key but bad for a (text, date) key, or
> whatever.

I don't think that's true, unless you have really bad hash functions on
the the component hashes. A hash combine function can't really do
anything about badly hashed input, what you want is that it doesn't
*reduce* the quality of the hash by combining.

Greetings,

Andres Freund


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Tue, Oct 10, 2017 at 4:37 PM, amul sul <sulamul@gmail.com> wrote:
> On Tue, Oct 10, 2017 at 3:42 PM, Ashutosh Bapat
> <ashutosh.bapat@enterprisedb.com> wrote:
>> On Tue, Oct 10, 2017 at 3:32 PM, amul sul <sulamul@gmail.com> wrote:
>>
>>>> +                                    hash_part? true : key->parttypbyval[j],
>>>> +                                    key->parttyplen[j]);
>>>> parttyplen is the length of partition key attribute, whereas what you want here
>>>> is the length of type of modulus and remainder. Is that correct? Probably we
>>>> need some special handling wherever parttyplen and parttypbyval is used e.g. in
>>>> call to partition_bounds_equal() from build_joinrel_partition_info().
>>>>
>>>
>>> Unless I am missing something, I don't think we should worry about parttyplen
>>> because in the datumCopy() when the datatype is pass-by-value then typelen
>>> is ignored.
>>
>> That's true, but it's ugly, passing typbyvalue of one type and len of other.
>>
>
> How about the attached patch(0003)?
> Also, the dim variable is renamed to natts.

Probably we should move changes to partition_bounds_copy() in 0003 to
0001, whose name suggests so.

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Mon, Oct 16, 2017 at 2:36 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

>
> Probably we should move changes to partition_bounds_copy() in 0003 to
> 0001, whose name suggests so.
>

We can't do this, hash partition strategy is introduced by 0002. Sorry
for the noise.

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Fri, Oct 13, 2017 at 3:00 AM, Andres Freund <andres@anarazel.de> wrote:
> On 2017-10-12 17:27:52 -0400, Robert Haas wrote:
>> On Thu, Oct 12, 2017 at 4:20 PM, Andres Freund <andres@anarazel.de> wrote:
>> >> In other words, it's not utterly fixed in stone --- we invented
>> >> --load-via-partition-root primarily to cope with circumstances that
>> >> could change hash values --- but we sure don't want to be changing it
>> >> with any regularity, or for a less-than-excellent reason.
>> >
>> > Yea, that's what I expected. It'd probably good for somebody to run
>> > smhasher or such on the output of the combine function (or even better,
>> > on both the 32 and 64 bit variants) in that case.
>>
>> Not sure how that test suite works exactly, but presumably the
>> characteristics in practice will depend the behavior of the hash
>> functions used as input the combine function - so the behavior could
>> be good for an (int, int) key but bad for a (text, date) key, or
>> whatever.
>
> I don't think that's true, unless you have really bad hash functions on
> the the component hashes. A hash combine function can't really do
> anything about badly hashed input, what you want is that it doesn't
> *reduce* the quality of the hash by combining.
>

I tried to get suggested SMHasher[1] test result for the hash_combine
for 32-bit and 64-bit version.

SMHasher works on hash keys of the form {0}, {0,1}, {0,1,2}... up to
N=255, using 256-N as the seed, for the hash_combine testing we
needed two hash value to be combined, for that, I've generated 64
and 128-bit hash using cityhash functions[2] for the given smhasher
key then split in two part to test 32-bit and 64-bit hash_combine
function respectively.   Attached patch for SMHasher code changes &
output of 32-bit and 64-bit hash_combine testing. Note that I have
skipped speed test this test which is irrelevant here.

By referring other hash function results [3], we can see that hash_combine
test results are not bad either.

Do let me know if current testing is not good enough or if you want me to do
more testing, thanks.

1] https://github.com/aappleby/smhasher
2] https://github.com/aappleby/smhasher/blob/master/src/CityTest.cpp
3] https://github.com/rurban/smhasher/tree/master/doc

Regards,
Amul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Thu, Oct 12, 2017 at 6:38 PM, amul sul <sulamul@gmail.com> wrote:
> On Thu, Oct 12, 2017 at 6:31 AM, Robert Haas <robertmhaas@gmail.com> wrote:
>> On Tue, Oct 10, 2017 at 7:07 AM, amul sul <sulamul@gmail.com> wrote:
>>> How about the attached patch(0003)?
>>> Also, the dim variable is renamed to natts.
>>
>> I'm not sure I believe this comment:
>>
>> +        /*
>> +         * We arrange the partitions in the ascending order of their modulus
>> +         * and remainders.  Also every modulus is factor of next larger
>> +         * modulus.  This means that the index of a given partition is same as
>> +         * the remainder of that partition.  Also entries at (remainder + N *
>> +         * modulus) positions in indexes array are all same for (modulus,
>> +         * remainder) specification for any partition.  Thus datums array from
>> +         * both the given bounds are same, if and only if their indexes array
>> +         * will be same.  So, it suffices to compare indexes array.
>> +         */
>>
>> I am particularly not sure that I believe that the index of a
>> partition must be the same as the remainder.  It doesn't seem like
>> that would be true when there is more than one modulus or when some
>> partitions are missing.
>>
>
> Looks like an explanation by the comment is not good enough, will think on this.
>
> Here are the links for the previous discussion:
> 1] https://postgr.es/m/CAFjFpRfHqSGBjNgJV2p%2BC4Yr5Qxvwygdsg4G_VQ6q9NTB-i3MA%40mail.gmail.com
> 2] https://postgr.es/m/CAFjFpRdeESKFkVGgmOdYvmD3d56-58c5VCBK0zDRjHrkq_VcNg%40mail.gmail.com
>
I have modified the comment little bit, now let me explain the theory behind it.

rd_partdesc->boundinfo->indexes array stores an index in rd_partdesc->oids
array corresponding to a given partition falls at the positions. And position in
indexes array is decided using remainder + N * modulus_of_that_partition
(where N = 0,1,2,..,).

For the case where the same modulus, the remainder will be 0,1,2,..,
and the index of that partition will be at 0,1,2,..,. (N=0).

For the case where more than one modulus then an index of a partition oid in the
oids array could be stored at the multiple places in indexes array if
its modulus is < greatest_modulus amongst bound (where N = 0,1,2,..,).

For example, partition bound (Modulus, remainder) = p1(2,0), p2(4,1),
p3(8,3), p4(8,7) Oids array [p1,p2,p3,p4] sorted by Modulus and then
by remainder and indexes array [0, 1, 0, 3, 0, 1, 0, 4] size of indexes
array is greatest_modulus.

In other word, if a partition index in oids array in the indexes array is
stored multiple times, then the lowest of the differences between them
is the modulus of that partition.  In above case for the partition p1, index
in oids array stored at 0,2,4,6. You can see lowest is the remainder and
minimum difference is the modulus of p1.

Since indexes arrays in both the bounds are same, for a given index in oids
array, the positions where it falls is same for both bounds. One can argue that
two different moduli could have the same remainder position, which is
not allowed
because that will cause partition overlap error at creation and also we have a
restriction on modulus that each modulus in the hash partition bound should be
the factor of next modulus.

> [....]
>
>> +static uint64
>> +mix_hash_value(int nkeys, Datum *hash_array, bool *isnull)
>>
> How about combining high 32 bits and the low 32 bits separately as shown below?
>
> static inline uint64
> hash_combine64(uint64 a, uint64 b)
> {
>     return (((uint64) hash_combine((uint32) a >> 32, (uint32) b >> 32) << 32)
>             | hash_combine((unit32) a, (unit32) b));
> }
>
I have used hash_combine64 function suggested by Andres [1].

>[....]
>> Have you checked how well the tests you've added cover the code you've
>> added?  What code is not covered by the tests, and is there any way to
>> cover it?
>>
> Will try to get gcov report for this patch.
>
Tests in the attached patch covers almost all the code expect few[2].

Updated patch attached.

1] https://postgr.es/m/20171012194353.3nealiykmjura4bi%40alap3.anarazel.de
2] Refer gcov_output.txt attachment.

Regards,
Amul Sul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Andres Freund
Date:
On 2017-10-24 12:43:12 +0530, amul sul wrote:
> I tried to get suggested SMHasher[1] test result for the hash_combine
> for 32-bit and 64-bit version.
> 
> SMHasher works on hash keys of the form {0}, {0,1}, {0,1,2}... up to
> N=255, using 256-N as the seed, for the hash_combine testing we
> needed two hash value to be combined, for that, I've generated 64
> and 128-bit hash using cityhash functions[2] for the given smhasher
> key then split in two part to test 32-bit and 64-bit hash_combine
> function respectively.   Attached patch for SMHasher code changes &
> output of 32-bit and 64-bit hash_combine testing. Note that I have
> skipped speed test this test which is irrelevant here.
> 
> By referring other hash function results [3], we can see that hash_combine
> test results are not bad either.
> 
> Do let me know if current testing is not good enough or if you want me to do
> more testing, thanks.

This looks very good! Both the tests you did, and the results for
hash_combineXX. I therefore think we can go ahead with that formulation
of hash_combine64?

Greetings,

Andres Freund


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Tue, Oct 24, 2017 at 5:00 PM, Andres Freund <andres@anarazel.de> wrote:
> On 2017-10-24 12:43:12 +0530, amul sul wrote:
>> I tried to get suggested SMHasher[1] test result for the hash_combine
>> for 32-bit and 64-bit version.
>>
>> SMHasher works on hash keys of the form {0}, {0,1}, {0,1,2}... up to
>> N=255, using 256-N as the seed, for the hash_combine testing we
>> needed two hash value to be combined, for that, I've generated 64
>> and 128-bit hash using cityhash functions[2] for the given smhasher
>> key then split in two part to test 32-bit and 64-bit hash_combine
>> function respectively.   Attached patch for SMHasher code changes &
>> output of 32-bit and 64-bit hash_combine testing. Note that I have
>> skipped speed test this test which is irrelevant here.
>>
>> By referring other hash function results [3], we can see that hash_combine
>> test results are not bad either.
>>
>> Do let me know if current testing is not good enough or if you want me to do
>> more testing, thanks.
>
> This looks very good! Both the tests you did, and the results for
> hash_combineXX. I therefore think we can go ahead with that formulation
> of hash_combine64?
>

Thanks, Andres. Yes we can, I've added your suggested hash_combine64 in
the latest patch[1].

Regards,
Amul

1] https://postgr.es/m/CAAJ_b97R2rJinGPAVmZZzpNV%3D-5BgYFxDfY9HYdM1bCYJFGmQw%40mail.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

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Tue, Oct 24, 2017 at 1:21 PM, amul sul <sulamul@gmail.com> wrote:
> Updated patch attached.

This patch needs a rebase.

It appears that satisfies_hash_func is declared incorrectly in
pg_proc.h.  ProcedureCreate seems to think that provariadic should be
ANYOID if the type of the last element is ANYOID, ANYELEMENTOID if the
type of the last element is ANYARRAYOID, and otherwise the element
type corresponding to the array type.   But here you have the last
element as int4[] but provariadic is any.  I wrote the following query
to detect problems of this type, and I think we might want to just go
ahead and add this to the regression test suite, verifying that it
returns no rows:

select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
from pg_proc where provariadic != 0
and case proargtypes[array_length(proargtypes, 1)-1]   when 2276 then 2276 -- any -> any   when 2277 then 2283 --
anyarray-> anyelement   else (select t.oid from pg_type t where t.typarray =
 
proargtypes[array_length(proargtypes, 1)-1]) end   != provariadic;

The simple fix is change provariadic to int4 and call it good.  It's
tempting to go the other way and actually make it
satisfies_hash_partition(int4, int4, variadic "any"), passing the
column values directly and letting satisfies_hash_partition doing the
hashing itself.  Any arguments that had a partition key type different
from the column type would have a RelabelType node placed on top of
the column, so that get_fn_expr_argtype would return the partition key
type.  Then, the function could look up the hash function for that
type and call it directly on the value.  That way, we'd be doing only
one function call instead of many, and the partition constraint would
look nicer in \d+ output, too.  :-)  On the other hand, that would
also mean that we'd have to look up the extended hash function every
time through this function, though maybe that could be prevented by
using fn_extra to cache FmgrInfos for all the hash functions on the
first time through.  I'm not sure how that would compare in terms of
speed with what you have now, but maybe it's worth trying.

The second paragraph of the CREATE TABLE documentation for PARTITION
OF needs to be updated like this: "The form with <literal>IN</literal>
is used for list partitioning, the form with <literal>FROM</literal>
and <literal>TO</literal> is used for range partitioning, and the form
with <literal>WITH</literal> is used for hash partitioning."

The CREATE TABLE documentation says "When using range partitioning,
the partition key can include multiple columns or expressions (up to
32,"; this should be changed to say "When using range or hash
partitioning".

-      expression.  If no B-tree operator class is specified when creating a
-      partitioned table, the default B-tree operator class for the
datatype will
-      be used.  If there is none, an error will be reported.
+      expression.  If no operator class is specified when creating a
partitioned
+      table, the default operator class of the appropriate type (btree for list
+      and range partitioning, hash for hash partitioning) will be used.  If
+      there is none, an error will be reported.
+     </para>
+
+     <para>
+      Since hash operator class provides only equality, not ordering, collation
+      is not relevant for hash partitioning. The behaviour will be unaffected
+      if a collation is specified.
+     </para>
+
+     <para>
+      Hash partitioning will use support function 2 routines from the operator
+      class. If there is none, an error will be reported.  See <xref
+      linkend="xindex-support"> for details of operator class support
+      functions.

I think we should rework this a little more heavily.  I suggest the
following, starting after "a single column or expression":

<para>
Range and list partitioning require a btree operator class, while hash
partitioning requires a hash operator class.  If no operator class is
specified explicitly, the default operator class of the appropriate
type will be used; if no default operator class exists, an error will
be raised.  When hash partitioning is used, the operator class used
must implement support function 2 (see <xref linkend="xindex-support">
for details).
</para>

I think we can leave out the part about collations.  It's possibly
worth a longer explanation here at some point: for range partitioning,
collation can affect which rows go into which partitions; for list
partitioning, it can't, but it can affect the order in which
partitions are expanded (which is a can of worms I'm not quite ready
to try to explain in user-facing documentation); for hash
partitioning, it makes no difference at all.  Although at some point
we may want to document this, I think it's a job for a separate patch,
since (1) the existing documentation doesn't document the precise
import of collations on existing partitioning types and (2) I'm not
sure that CREATE TABLE is really the best place to explain this.

The example commands for creating a hash-partitioned table are missing
spaces between WITH and the parenthesis which follows.

In 0003, the changes to partition_bounds_copy claim that I shouldn't
worry about the fact that typlen is set to 4 because datumCopy won't
use it for a pass-by-value datatype, but I think that calling
functions with incorrect arguments and hoping that they ignore them
and therefore nothing bad happens doesn't sound like a very good idea.
Fortunately, I think the actual code is fine; I think we just need to
change the comments.  For hash partitioning, the datums array always
contains two integers, which are of type int4, which is indeed a
pass-by-value type of length 4 (note that if we were using int8 for
the modulus and remainder, we'd need to set byval to FLOAT8PASSBYVAL).
I would just write this as:

if (hash_part)
{   typlen = sizeof(int32); /* always int4 */   byval = true;           /* int4 is pass-by-value */
}

+       for (i = 0; i < nkeys; i++)
+       {
+               if (!isnull[i])
+                       rowHash = hash_combine64(rowHash,
DatumGetUInt64(hash_array[i]));
+       }

Excess braces.

I think it might be better to inline the logic in mix_hash_value()
into each of the two callers.  Then, the callers wouldn't need Datum
hash_array[PARTITION_MAX_KEYS]; they could just fold each new hash
value into a uint64 value.  That seems likely to be slightly faster
and I don't see any real downside.

rhaas=# create table natch (a citext, b text) partition by hash (a);
ERROR:  XX000: missing support function 2(16398,16398) in opfamily 16437
LOCATION:  RelationBuildPartitionKey, relcache.c:954

It shouldn't be possible to reach an elog() from SQL, and this is not
a friendly error message.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
 On Sun, Oct 29, 2017 at 12:38 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> On Tue, Oct 24, 2017 at 1:21 PM, amul sul <sulamul@gmail.com> wrote:
>> Updated patch attached.
>
> This patch needs a rebase.

Sure, thanks a lot for your review.

>
> It appears that satisfies_hash_func is declared incorrectly in
> pg_proc.h.  ProcedureCreate seems to think that provariadic should be
> ANYOID if the type of the last element is ANYOID, ANYELEMENTOID if the
> type of the last element is ANYARRAYOID, and otherwise the element
> type corresponding to the array type.   But here you have the last
> element as int4[] but provariadic is any.

Actually, int4[] is also inappropriate type as we have started using a 64bit
hash function.  We need something int8[] which is not available, so that I
have used ANYARRAYOID in the attached patch(0004).

> I wrote the following query
> to detect problems of this type, and I think we might want to just go
> ahead and add this to the regression test suite, verifying that it
> returns no rows:
>
> select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
> from pg_proc where provariadic != 0
> and case proargtypes[array_length(proargtypes, 1)-1]
>     when 2276 then 2276 -- any -> any
>     when 2277 then 2283 -- anyarray -> anyelement
>     else (select t.oid from pg_type t where t.typarray =
> proargtypes[array_length(proargtypes, 1)-1]) end
>     != provariadic;
>

Added in 0001 patch.

> The simple fix is change provariadic to int4 and call it good.  It's
> tempting to go the other way and actually make it
> satisfies_hash_partition(int4, int4, variadic "any"), passing the
> column values directly and letting satisfies_hash_partition doing the
> hashing itself.  Any arguments that had a partition key type different
> from the column type would have a RelabelType node placed on top of
> the column, so that get_fn_expr_argtype would return the partition key
> type.  Then, the function could look up the hash function for that
> type and call it directly on the value.  That way, we'd be doing only
> one function call instead of many, and the partition constraint would
> look nicer in \d+ output, too.  :-)  On the other hand, that would
> also mean that we'd have to look up the extended hash function every
> time through this function, though maybe that could be prevented by
> using fn_extra to cache FmgrInfos for all the hash functions on the
> first time through.  I'm not sure how that would compare in terms of
> speed with what you have now, but maybe it's worth trying.
>

One advantage of current implementation is that we can see which hash
function are used for the each partitioning column and also we don't need to
worry about user specified opclass and different input types.

Something similar I've tried in my initial patch version[1], but I have missed
user specified opclass handling for each partitioning column.  Do you want me
to handle opclass using RelabelType node? I am afraid that, that would make
the \d+ output more horrible than the current one if non-default opclass used.

> The second paragraph of the CREATE TABLE documentation for PARTITION
> OF needs to be updated like this: "The form with <literal>IN</literal>
> is used for list partitioning, the form with <literal>FROM</literal>
> and <literal>TO</literal> is used for range partitioning, and the form
> with <literal>WITH</literal> is used for hash partitioning."
>

Fixed in the attached version(0004).

> The CREATE TABLE documentation says "When using range partitioning,
> the partition key can include multiple columns or expressions (up to
> 32,"; this should be changed to say "When using range or hash
> partitioning".
>

Fixed in the attached version(0004).

> -      expression.  If no B-tree operator class is specified when creating a
> -      partitioned table, the default B-tree operator class for the
> datatype will
> -      be used.  If there is none, an error will be reported.
> +      expression.  If no operator class is specified when creating a
> partitioned
> +      table, the default operator class of the appropriate type (btree for list
> +      and range partitioning, hash for hash partitioning) will be used.  If
> +      there is none, an error will be reported.
> +     </para>
> +
> +     <para>
> +      Since hash operator class provides only equality, not ordering, collation
> +      is not relevant for hash partitioning. The behaviour will be unaffected
> +      if a collation is specified.
> +     </para>
> +
> +     <para>
> +      Hash partitioning will use support function 2 routines from the operator
> +      class. If there is none, an error will be reported.  See <xref
> +      linkend="xindex-support"> for details of operator class support
> +      functions.
>
> I think we should rework this a little more heavily.  I suggest the
> following, starting after "a single column or expression":
>
> <para>
> Range and list partitioning require a btree operator class, while hash
> partitioning requires a hash operator class.  If no operator class is
> specified explicitly, the default operator class of the appropriate
> type will be used; if no default operator class exists, an error will
> be raised.  When hash partitioning is used, the operator class used
> must implement support function 2 (see <xref linkend="xindex-support">
> for details).
> </para>
>

Thanks again, added in the attached version(0004).

> I think we can leave out the part about collations.  It's possibly
> worth a longer explanation here at some point: for range partitioning,
> collation can affect which rows go into which partitions; for list
> partitioning, it can't, but it can affect the order in which
> partitions are expanded (which is a can of worms I'm not quite ready
> to try to explain in user-facing documentation); for hash
> partitioning, it makes no difference at all.  Although at some point
> we may want to document this, I think it's a job for a separate patch,
> since (1) the existing documentation doesn't document the precise
> import of collations on existing partitioning types and (2) I'm not
> sure that CREATE TABLE is really the best place to explain this.
>

Okay.

> The example commands for creating a hash-partitioned table are missing
> spaces between WITH and the parenthesis which follows.
>

Fixed in the attached version(0004).

> In 0003, the changes to partition_bounds_copy claim that I shouldn't
> worry about the fact that typlen is set to 4 because datumCopy won't
> use it for a pass-by-value datatype, but I think that calling
> functions with incorrect arguments and hoping that they ignore them
> and therefore nothing bad happens doesn't sound like a very good idea.
> Fortunately, I think the actual code is fine; I think we just need to
> change the comments.  For hash partitioning, the datums array always
> contains two integers, which are of type int4, which is indeed a
> pass-by-value type of length 4 (note that if we were using int8 for
> the modulus and remainder, we'd need to set byval to FLOAT8PASSBYVAL).
> I would just write this as:
>
> if (hash_part)
> {
>     typlen = sizeof(int32); /* always int4 */
>     byval = true;           /* int4 is pass-by-value */
> }
>

Fixed in the attached version (now patch number is 0005).

> +       for (i = 0; i < nkeys; i++)
> +       {
> +               if (!isnull[i])
> +                       rowHash = hash_combine64(rowHash,
> DatumGetUInt64(hash_array[i]));
> +       }
>
> Excess braces.
>

Fixed in the attached version(0004).

> I think it might be better to inline the logic in mix_hash_value()
> into each of the two callers.  Then, the callers wouldn't need Datum
> hash_array[PARTITION_MAX_KEYS]; they could just fold each new hash
> value into a uint64 value.  That seems likely to be slightly faster
> and I don't see any real downside.
>

Fixed in the attached version(0004).

> rhaas=# create table natch (a citext, b text) partition by hash (a);
> ERROR:  XX000: missing support function 2(16398,16398) in opfamily 16437
> LOCATION:  RelationBuildPartitionKey, relcache.c:954
>
> It shouldn't be possible to reach an elog() from SQL, and this is not
> a friendly error message.
>

How about an error message in the attached patch(0004)?


1] https://postgr.es/m/CAAJ_b96AQBAxSQ2mxnTmx9zXh79GdP_dQWv0aupjcmz+jpiGjw@mail.gmail.com

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

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Mon, Oct 30, 2017 at 5:52 PM, amul sul <sulamul@gmail.com> wrote:
> Actually, int4[] is also inappropriate type as we have started using a 64bit
> hash function.  We need something int8[] which is not available, so that I
> have used ANYARRAYOID in the attached patch(0004).

I don't know why you think int8[] is not available.

rhaas=# select 'int8[]'::regtype;regtype
----------bigint[]
(1 row)

>> I wrote the following query
>> to detect problems of this type, and I think we might want to just go
>> ahead and add this to the regression test suite, verifying that it
>> returns no rows:
>>
>> select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
>> from pg_proc where provariadic != 0
>> and case proargtypes[array_length(proargtypes, 1)-1]
>>     when 2276 then 2276 -- any -> any
>>     when 2277 then 2283 -- anyarray -> anyelement
>>     else (select t.oid from pg_type t where t.typarray =
>> proargtypes[array_length(proargtypes, 1)-1]) end
>>     != provariadic;
>>
>
> Added in 0001 patch.

Committed.

> One advantage of current implementation is that we can see which hash
> function are used for the each partitioning column and also we don't need to
> worry about user specified opclass and different input types.
>
> Something similar I've tried in my initial patch version[1], but I have missed
> user specified opclass handling for each partitioning column.  Do you want me
> to handle opclass using RelabelType node? I am afraid that, that would make
> the \d+ output more horrible than the current one if non-default opclass used.

Maybe we should just pass the OID of the partition (or both the
partition and the parent, so we can get the lock ordering right?)
instead.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Tue, Oct 31, 2017 at 9:54 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> On Mon, Oct 30, 2017 at 5:52 PM, amul sul <sulamul@gmail.com> wrote:
>> Actually, int4[] is also inappropriate type as we have started using a 64bit
>> hash function.  We need something int8[] which is not available, so that I
>> have used ANYARRAYOID in the attached patch(0004).
>
> I don't know why you think int8[] is not available.
>
> rhaas=# select 'int8[]'::regtype;
>  regtype
> ----------
>  bigint[]
> (1 row)
>

I missed _int8, was searching for INT8ARRAYOID in pg_type.h, my bad.

>>> I wrote the following query
>>> to detect problems of this type, and I think we might want to just go
>>> ahead and add this to the regression test suite, verifying that it
>>> returns no rows:
>>>
>>> select oid::regprocedure, provariadic::regtype, proargtypes::regtype[]
>>> from pg_proc where provariadic != 0
>>> and case proargtypes[array_length(proargtypes, 1)-1]
>>>     when 2276 then 2276 -- any -> any
>>>     when 2277 then 2283 -- anyarray -> anyelement
>>>     else (select t.oid from pg_type t where t.typarray =
>>> proargtypes[array_length(proargtypes, 1)-1]) end
>>>     != provariadic;
>>>
>>
>> Added in 0001 patch.
>
> Committed.
>

Thanks !

>> One advantage of current implementation is that we can see which hash
>> function are used for the each partitioning column and also we don't need to
>> worry about user specified opclass and different input types.
>>
>> Something similar I've tried in my initial patch version[1], but I have missed
>> user specified opclass handling for each partitioning column.  Do you want me
>> to handle opclass using RelabelType node? I am afraid that, that would make
>> the \d+ output more horrible than the current one if non-default opclass used.
>
> Maybe we should just pass the OID of the partition (or both the
> partition and the parent, so we can get the lock ordering right?)
> instead.
>
Okay, will try this.


Regards,
Amul


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Tue, Oct 31, 2017 at 10:17 AM, amul sul <sulamul@gmail.com> wrote:
> On Tue, Oct 31, 2017 at 9:54 AM, Robert Haas <robertmhaas@gmail.com> wrote:
>> On Mon, Oct 30, 2017 at 5:52 PM, amul sul <sulamul@gmail.com> wrote:
>>> Actually, int4[] is also inappropriate type as we have started using a 64bit
>>> hash function.  We need something int8[] which is not available, so that I
>>> have used ANYARRAYOID in the attached patch(0004).
>>
>> I don't know why you think int8[] is not available.
>>
>> rhaas=# select 'int8[]'::regtype;
>>  regtype
>> ----------
>>  bigint[]
>> (1 row)
>>
>
> I missed _int8, was searching for INT8ARRAYOID in pg_type.h, my bad.
>

Fixed in the 0003 patch.

>>>>[....]
>>> Something similar I've tried in my initial patch version[1], but I have missed
>>> user specified opclass handling for each partitioning column.  Do you want me
>>> to handle opclass using RelabelType node? I am afraid that, that would make
>>> the \d+ output more horrible than the current one if non-default opclass used.
>>
>> Maybe we should just pass the OID of the partition (or both the
>> partition and the parent, so we can get the lock ordering right?)
>> instead.
>>
> Okay, will try this.
>

In 0005, I rewrote satisfies_hash_partition, to accept parent id, modulus and
remainder as before, and the column values directly. This function opens parent
relation to get its PartitionKey which has extended hash function information in
a partsupfunc array, using this it will calculates a hash for the partition key.
Also, it will copy this partsupfunc array into function memory context so that
we don't need to open parent relation again and again in the subsequent function
call to get extended hash functions information (e.g. bulk insert).

In \d+ partition constraint will be :
satisfies_hash_partition('16384'::oid, 2, 0, a, b)
where 16384 is parent relid, 2 is modulus, 0 is remainder and 'a' &
'b' are partition
column.

In the earlier version partition constraint was (i.e. without 0005 patch):
satisfies_hash_partition(2, 0,
hashint4extended(a,'8816678312871386365'::bigint),
                         hashtextextended(b, '8816678312871386365'::bigint))


I did small performance test using a copy command to load 100,000,000 records
and a separate insert command for each record to load 2,00,000 records and
result are as follow:

+---------+-----------------+--------------------+
| Command | With 0005 patch | Without 0005 patch |
+---------+-----------------+--------------------+
| COPY    | 63.719 seconds  | 64.925 seconds     |
+---------+-----------------+--------------------+
| INSERT  | 179.21 seconds  | 174.89 seconds     |
+---------+-----------------+--------------------+

Although partition constraints become more simple, there isn't any performance
gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
copied extended hash function info from the partition key, what if parent is
changed while we are using it? Do we need to keep lock on parent until commit in
satisfies_hash_partition?

Regards,
Amul

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Wed, Nov 1, 2017 at 3:46 PM, amul sul <sulamul@gmail.com> wrote:
> Although partition constraints become more simple, there isn't any performance
> gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
> copied extended hash function info from the partition key, what if parent is
> changed while we are using it? Do we need to keep lock on parent until commit in
> satisfies_hash_partition?

I don't think it should be possible for the parent to be changed.  I
mean, the partition key is altogether immutable -- it can't be changed
after creation time.  The partition bounds can be changed for
individual partitions but that would require a lock on the partition.

Can you give an example of the kind of scenario about which you are concerned?

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Thu, Nov 2, 2017 at 1:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> On Wed, Nov 1, 2017 at 3:46 PM, amul sul <sulamul@gmail.com> wrote:
>> Although partition constraints become more simple, there isn't any performance
>> gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
>> copied extended hash function info from the partition key, what if parent is
>> changed while we are using it? Do we need to keep lock on parent until commit in
>> satisfies_hash_partition?
>
> I don't think it should be possible for the parent to be changed.  I
> mean, the partition key is altogether immutable -- it can't be changed
> after creation time.  The partition bounds can be changed for
> individual partitions but that would require a lock on the partition.
>
> Can you give an example of the kind of scenario about which you are concerned?
>

Yes, you are correct, column involved in the partitioning are immutable.

I was just worried about any change in the partition key column that
might change selected hash function.

Regards,
Amul


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Ashutosh Bapat
Date:
On Thu, Nov 2, 2017 at 1:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> On Wed, Nov 1, 2017 at 3:46 PM, amul sul <sulamul@gmail.com> wrote:
>> Although partition constraints become more simple, there isn't any performance
>> gain with 0005 patch. Also I am little skeptic about logic in 0005 where we
>> copied extended hash function info from the partition key, what if parent is
>> changed while we are using it? Do we need to keep lock on parent until commit in
>> satisfies_hash_partition?
>
> I don't think it should be possible for the parent to be changed.  I
> mean, the partition key is altogether immutable -- it can't be changed
> after creation time.  The partition bounds can be changed for
> individual partitions but that would require a lock on the partition.
>
> Can you give an example of the kind of scenario about which you are concerned?

Right now partition keys are immutable but we don't have much code
written with that assumption. All the code usually keeps a lock on the
parent till the time they use the information in the partition key. In
a distant future, which may not exist, we may support ALTER TABLE ...
PARTITION BY to change partition keys (albeit at huge cost of data
movement). If we do that, we will have to remember this one-off
instance of code which assumes that the partition keys are immutable.

-- 
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Nov 2, 2017 at 1:45 PM, amul sul <sulamul@gmail.com> wrote:
> Yes, you are correct, column involved in the partitioning are immutable.
>
> I was just worried about any change in the partition key column that
> might change selected hash function.

Any such change, even if it were allowed, would have to take
AccessExclusiveLock on the child.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Thu, Nov 2, 2017 at 1:52 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
> Right now partition keys are immutable but we don't have much code
> written with that assumption. All the code usually keeps a lock on the
> parent till the time they use the information in the partition key. In
> a distant future, which may not exist, we may support ALTER TABLE ...
> PARTITION BY to change partition keys (albeit at huge cost of data
> movement). If we do that, we will have to remember this one-off
> instance of code which assumes that the partition keys are immutable.

I am pretty sure this is by no means the only piece of code which assumes that.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Re: [HACKERS] [POC] hash partitioning

From
Robert Haas
Date:
On Wed, Nov 1, 2017 at 6:16 AM, amul sul <sulamul@gmail.com> wrote:
> Fixed in the 0003 patch.

I have committed this patch set with the attached adjustments.

-- 
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Attachment

Re: [HACKERS] [POC] hash partitioning

From
amul sul
Date:
On Fri, Nov 10, 2017 at 4:41 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> On Wed, Nov 1, 2017 at 6:16 AM, amul sul <sulamul@gmail.com> wrote:
>> Fixed in the 0003 patch.
>
> I have committed this patch set with the attached adjustments.
>

Thanks a lot for your support & a ton of thanks to all reviewer.

Regards,
Amul


-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers