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;
 
