Re: [HACKERS] [POC] hash partitioning

Поиск
Список
Период
Сортировка
От Aleksander Alekseev
Тема Re: [HACKERS] [POC] hash partitioning
Дата
Msg-id 20170228150536.GB19777@e733.localdomain
обсуждение исходный текст
Ответ на [HACKERS] [POC] hash partitioning  (Yugo Nagata <nagata@sraoss.co.jp>)
Ответы Re: [HACKERS] [POC] hash partitioning  (Yugo Nagata <nagata@sraoss.co.jp>)
Список pgsql-hackers
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

В списке pgsql-hackers по дате отправления:

Предыдущее
От: Andreas Karlsson
Дата:
Сообщение: Re: [HACKERS] Disallowing multiple queries per PQexec()
Следующее
От: Andrew Dunstan
Дата:
Сообщение: Re: [HACKERS] timeouts in PostgresNode::psql