From 7dde519faee8fdf41f3734f25b17a4ab434389d4 Mon Sep 17 00:00:00 2001
From: Amul Sul <sulamul@gmail.com>
Date: Thu, 2 Mar 2017 13:32:34 +0530
Subject: [PATCH] Declarative hash partitioning

---
 src/backend/catalog/partition.c            | 569 ++++++++++++++++++++++++++---
 src/backend/commands/tablecmds.c           |  63 +++-
 src/backend/nodes/copyfuncs.c              |   2 +
 src/backend/nodes/equalfuncs.c             |   2 +
 src/backend/nodes/outfuncs.c               |   2 +
 src/backend/nodes/readfuncs.c              |   2 +
 src/backend/parser/gram.y                  |  46 ++-
 src/backend/parser/parse_utilcmd.c         |  28 +-
 src/backend/utils/adt/ruleutils.c          |  10 +
 src/include/catalog/pg_proc.h              |   3 +
 src/include/nodes/parsenodes.h             |   8 +-
 src/test/regress/expected/alter_table.out  | 231 ++++++++++++
 src/test/regress/expected/create_table.out |  51 ++-
 src/test/regress/expected/insert.out       |  36 ++
 src/test/regress/expected/update.out       |  21 ++
 src/test/regress/sql/alter_table.sql       | 191 ++++++++++
 src/test/regress/sql/create_table.sql      |  37 +-
 src/test/regress/sql/insert.sql            |  30 ++
 src/test/regress/sql/update.sql            |  19 +
 19 files changed, 1264 insertions(+), 87 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 4bcef58..ba1c727 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -48,6 +48,7 @@
 #include "utils/rel.h"
 #include "utils/ruleutils.h"
 #include "utils/syscache.h"
+#include "utils/typcache.h"
 
 /*
  * Information about bounds of a partitioned relation
@@ -76,16 +77,18 @@ typedef enum RangeDatumContent
 
 typedef struct PartitionBoundInfoData
 {
-	char		strategy;		/* list or range bounds? */
+	char		strategy;		/* hash, list or range bounds? */
 	int			ndatums;		/* Length of the datums following array */
 	Datum	  **datums;			/* Array of datum-tuples with key->partnatts
 								 * datums each */
 	RangeDatumContent **content;/* what's contained in each range bound datum?
 								 * (see the above enum); NULL for list
 								 * partitioned tables */
-	int		   *indexes;		/* Partition indexes; one entry per member of
-								 * the datums array (plus one if range
-								 * partitioned table) */
+	int		   *indexes;		/* Partition indexes; in case of hash
+								 * partitioned table array length will be
+								 * value of largest modulus, and for others
+								 * one entry per member of the datums array
+								 * (plus one if range partitioned table) */
 	bool		has_null;		/* Is there a null-accepting partition? false
 								 * for range partitioned tables */
 	int			null_index;		/* Index of the null-accepting partition; -1
@@ -97,6 +100,14 @@ typedef struct PartitionBoundInfoData
  * is represented with one of the following structs.
  */
 
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+	int		modulus;
+	int		remainder;
+	int		index;
+} PartitionHashBound;
+
 /* One value coming from some (index'th) list partition */
 typedef struct PartitionListValue
 {
@@ -113,11 +124,13 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
 static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
 							   void *arg);
 static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
 						   void *arg);
 
+static List *get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec);
 static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
 static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
 static Oid get_partition_operator(PartitionKey key, int col,
@@ -147,6 +160,11 @@ static void FormPartitionKeyDatum(PartitionDispatch pd,
 					  Datum *values,
 					  bool *isnull);
 
+static uint32 cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values,
+							 bool *isnull);
+/* SQL-callable function for use in hash partition CHECK constraints */
+PG_FUNCTION_INFO_V1(satisfies_hash_partition);
+
 /*
  * RelationBuildPartitionDesc
  *		Form rel's partition descriptor
@@ -170,6 +188,9 @@ RelationBuildPartitionDesc(Relation rel)
 
 	int			ndatums = 0;
 
+	/* Hash partitioning specific */
+	PartitionHashBound **hbounds = NULL;
+
 	/* List partitioning specific */
 	PartitionListValue **all_values = NULL;
 	bool		found_null = false;
@@ -236,7 +257,33 @@ RelationBuildPartitionDesc(Relation rel)
 			oids[i++] = lfirst_oid(cell);
 
 		/* Convert from node to the internal representation */
-		if (key->strategy == PARTITION_STRATEGY_LIST)
+		if (key->strategy == PARTITION_STRATEGY_HASH)
+		{
+			ndatums = nparts;
+			hbounds = (PartitionHashBound **) palloc(nparts *
+													 sizeof(PartitionHashBound *));
+			i = 0;
+			foreach (cell, boundspecs)
+			{
+				PartitionBoundSpec *spec = lfirst(cell);
+
+				if (spec->strategy != PARTITION_STRATEGY_HASH)
+					elog(ERROR, "invalid strategy in partition bound spec");
+
+				hbounds[i] = (PartitionHashBound *)
+					palloc(sizeof(PartitionHashBound));
+
+				hbounds[i]->modulus = spec->modulus;
+				hbounds[i]->remainder = spec->remainder;
+				hbounds[i]->index = i;
+				i++;
+			}
+
+			/* Sort all the bounds in ascending order */
+			qsort(hbounds, nparts, sizeof(PartitionHashBound *),
+				  qsort_partition_hbound_cmp);
+		}
+		else if (key->strategy == PARTITION_STRATEGY_LIST)
 		{
 			List	   *non_null_values = NIL;
 
@@ -463,6 +510,42 @@ RelationBuildPartitionDesc(Relation rel)
 
 		switch (key->strategy)
 		{
+			case PARTITION_STRATEGY_HASH:
+				{
+					/* Modulus are stored in ascending order */
+					int	greatest_modulus = hbounds[ndatums - 1]->modulus;
+
+					boundinfo->indexes = (int *) palloc(greatest_modulus *
+														sizeof(int));
+					memset(boundinfo->indexes, -1,
+						   greatest_modulus * sizeof(int));
+
+					for (i = 0; i < nparts; i++)
+					{
+						int		mod = hbounds[i]->modulus,
+								place = hbounds[i]->remainder;
+
+						boundinfo->datums[i] = (Datum *) palloc(2 *
+																sizeof(Datum));
+						boundinfo->datums[i][0] = Int32GetDatum(mod);
+						boundinfo->datums[i][1] = Int32GetDatum(place);
+						next_index = hbounds[i]->index;
+
+						while (place < greatest_modulus)
+						{
+							/* overlap? */
+							Assert(boundinfo->indexes[place] == -1);
+							boundinfo->indexes[place] = next_index;
+							place = place + mod;
+						}
+
+						mapping[i] = i;
+						pfree(hbounds[i]);
+					}
+					pfree(hbounds);
+					break;
+				}
+
 			case PARTITION_STRATEGY_LIST:
 				{
 					boundinfo->has_null = found_null;
@@ -616,53 +699,77 @@ partition_bounds_equal(PartitionKey key,
 	if (b1->null_index != b2->null_index)
 		return false;
 
-	for (i = 0; i < b1->ndatums; i++)
+	if (key->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			j;
+		int greatest_modulus;
 
-		for (j = 0; j < key->partnatts; j++)
+		/*
+		 * Hash partition bound stores modulus and remainder at
+		 * b1->datums[i][0] and b1->datums[i][0] position respectively.
+		 */
+		for (i = 0; i < b1->ndatums; i++)
+			if (!(datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+							   true, sizeof(int)) &&
+				  datumIsEqual(b1->datums[i][0], b2->datums[i][0],
+							   true, sizeof(int))))
+				return false;
+
+		/* Compare indexes */
+		greatest_modulus = DatumGetInt32(b1->datums[b1->ndatums - 1][0]);
+		for (i = 0; i < greatest_modulus; i++)
+			if (b1->indexes[i] != b2->indexes[i])
+				return false;
+	}
+	else
+	{
+		for (i = 0; i < b1->ndatums; i++)
 		{
-			/* For range partitions, the bounds might not be finite. */
-			if (b1->content != NULL)
+			int			j;
+
+			for (j = 0; j < key->partnatts; j++)
 			{
+				/* For range partitions, the bounds might not be finite. */
+				if (b1->content != NULL)
+				{
+					/*
+					 * A finite bound always differs from an infinite bound, and
+					 * different kinds of infinities differ from each other.
+					 */
+					if (b1->content[i][j] != b2->content[i][j])
+						return false;
+
+					/* Non-finite bounds are equal without further examination. */
+					if (b1->content[i][j] != RANGE_DATUM_FINITE)
+						continue;
+				}
+
 				/*
-				 * A finite bound always differs from an infinite bound, and
-				 * different kinds of infinities differ from each other.
+				 * Compare the actual values. Note that it would be both incorrect
+				 * and unsafe to invoke the comparison operator derived from the
+				 * partitioning specification here.  It would be incorrect because
+				 * we want the relcache entry to be updated for ANY change to the
+				 * partition bounds, not just those that the partitioning operator
+				 * thinks are significant.  It would be unsafe because we might
+				 * reach this code in the context of an aborted transaction, and
+				 * an arbitrary partitioning operator might not be safe in that
+				 * context.  datumIsEqual() should be simple enough to be safe.
 				 */
-				if (b1->content[i][j] != b2->content[i][j])
+				if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
+								  key->parttypbyval[j],
+								  key->parttyplen[j]))
 					return false;
-
-				/* Non-finite bounds are equal without further examination. */
-				if (b1->content[i][j] != RANGE_DATUM_FINITE)
-					continue;
 			}
 
-			/*
-			 * Compare the actual values. Note that it would be both incorrect
-			 * and unsafe to invoke the comparison operator derived from the
-			 * partitioning specification here.  It would be incorrect because
-			 * we want the relcache entry to be updated for ANY change to the
-			 * partition bounds, not just those that the partitioning operator
-			 * thinks are significant.  It would be unsafe because we might
-			 * reach this code in the context of an aborted transaction, and
-			 * an arbitrary partitioning operator might not be safe in that
-			 * context.  datumIsEqual() should be simple enough to be safe.
-			 */
-			if (!datumIsEqual(b1->datums[i][j], b2->datums[i][j],
-							  key->parttypbyval[j],
-							  key->parttyplen[j]))
+			if (b1->indexes[i] != b2->indexes[i])
 				return false;
 		}
 
-		if (b1->indexes[i] != b2->indexes[i])
+		/* There are ndatums+1 indexes in case of range partitions */
+		if (key->strategy == PARTITION_STRATEGY_RANGE &&
+			b1->indexes[i] != b2->indexes[i])
 			return false;
 	}
 
-	/* There are ndatums+1 indexes in case of range partitions */
-	if (key->strategy == PARTITION_STRATEGY_RANGE &&
-		b1->indexes[i] != b2->indexes[i])
-		return false;
-
 	return true;
 }
 
@@ -684,6 +791,92 @@ check_new_partition_bound(char *relname, Relation parent, Node *bound)
 
 	switch (key->strategy)
 	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+				Assert(spec->remainder >= 0 && spec->remainder < spec->modulus);
+
+				if (partdesc->nparts > 0)
+				{
+					PartitionBoundInfo		boundinfo = partdesc->boundinfo;
+					Datum				  **datums = boundinfo->datums;
+					int						ndatums = boundinfo->ndatums;
+					int 					greatest_modulus;
+					int						place;
+
+					/*
+					 * Check rule that every modulus must be a factor of the
+					 * next larger modulus.  For example, if you have a bunch
+					 * of partitions that all have modulus 5, you can add a new
+					 * new partition with modulus 10 or a new partition with
+					 * modulus 15, but you cannot add both a partition with
+					 * modulus 10 and a partition with modulus 15, because 10
+					 * is not a factor of 15.  However, you could simultaneously
+					 * use modulus 4, modulus 8, modulus 16, and modulus 32 if
+					 * you wished, because each modulus is a factor of the next
+					 * larger one.  You could also use modulus 10, modulus 20,
+					 * and modulus 60. But you could not use modulus 10,
+					 * modulus 15, and modulus 60 for the same reason.
+					 */
+					{
+						int			offset;
+						bool		equal,
+									valid_bound = true;
+						int			pmod, /* Previous largest modulus */
+									nmod; /* Next largest modulus */
+
+						/*
+						 * Get greatest bound in array boundinfo->datums which
+						 * is less than or equal to spec->modulus and
+						 * spec->remainder
+						 */
+						offset = partition_bound_bsearch(key, boundinfo, spec,
+														 true, &equal);
+						if (offset < 0)
+						{
+							nmod = DatumGetInt32(datums[0][0]);
+							valid_bound = (nmod % spec->modulus) == 0;
+						}
+						else
+						{
+							pmod = DatumGetInt32(datums[offset][0]);
+							valid_bound = (spec->modulus % pmod) == 0;
+
+							if (valid_bound && (offset + 1) < ndatums)
+							{
+								nmod = DatumGetInt32(datums[offset + 1][0]);
+								valid_bound = (nmod % spec->modulus) == 0;
+							}
+						}
+
+						if (!valid_bound)
+							ereport(ERROR,
+									(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+									 errmsg("invalid bound specification for a hash partition"),
+									 errhint("every modulus must be factor of next largest modulus")));
+					}
+
+					greatest_modulus = DatumGetInt32(datums[ndatums - 1][0]);
+					place = spec->remainder;
+
+					if (place >= greatest_modulus)
+						place = place % greatest_modulus;
+
+					do
+					{
+						if (boundinfo->indexes[place] != -1)
+						{
+							overlap = true;
+							with = boundinfo->indexes[place];
+							break;
+						}
+						place = place + spec->modulus;
+					} while (place < greatest_modulus);
+				}
+
+				break;
+			}
+
 		case PARTITION_STRATEGY_LIST:
 			{
 				Assert(spec->strategy == PARTITION_STRATEGY_LIST);
@@ -906,6 +1099,11 @@ get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
 
 	switch (key->strategy)
 	{
+		case PARTITION_STRATEGY_HASH:
+			Assert(spec->strategy == PARTITION_STRATEGY_HASH);
+			my_qual = get_qual_for_hash(key, spec);
+			break;
+
 		case PARTITION_STRATEGY_LIST:
 			Assert(spec->strategy == PARTITION_STRATEGY_LIST);
 			my_qual = get_qual_for_list(key, spec);
@@ -1149,6 +1347,93 @@ RelationGetPartitionDispatchInfo(Relation rel, int lockmode,
 /* Module-local functions */
 
 /*
+ * get_qual_for_hash
+ *
+ * Given a list of partition columns, modulus and remainder this function
+ * returns an expression Node for the partition table's CHECK constraint.
+ *
+ * For example, given a partition definition such as:
+ *  CREATE TABLE simple_hash (pkey int, value char(10))
+ *  PARTITION BY HASH (pkey, value);
+ *
+ * CREATE TABLE p_p1 PARTITION OF simple_hash WITH (modulus 2, remainder 1);
+ * CREATE TABLE p_p2 PARTITION OF simple_hash WITH (modulus 4, remainder 2);
+ * CREATE TABLE p_p3 PARTITION OF simple_hash WITH (modulus 8, remainder 0);
+ * CREATE TABLE p_p4 PARTITION OF simple_hash WITH (modulus 8, remainder 4);
+ *
+ * This function will return one of the following in the form of a
+ * subexpression:
+ *
+ *   for p_p1: satisfies_hash_partition(2, 1, pkey, value)
+ *   for p_p2: satisfies_hash_partition(4, 2, pkey, value)
+ *   for p_p3: satisfies_hash_partition(8, 0, pkey, value)
+ *   for p_p4: satisfies_hash_partition(8, 4, pkey, value)
+ */
+static List *
+get_qual_for_hash(PartitionKey key, PartitionBoundSpec *spec)
+{
+	FuncExpr   *fexpr;
+	Node	   *modulusConst;
+	Node	   *remainderConst;
+	List	   *args;
+	ListCell   *partexprs_item;
+	int			i;
+
+	/* Fixed arguments. */
+	modulusConst = (Node *) makeConst(INT4OID,
+									  -1,
+									  InvalidOid,
+									  sizeof(int32),
+									  Int32GetDatum(spec->modulus),
+									  false,
+									  true);
+
+	remainderConst = (Node *) makeConst(INT4OID,
+										-1,
+										InvalidOid,
+										sizeof(int32),
+										Int32GetDatum(spec->remainder),
+										false,
+										true);
+
+	args = list_make2(modulusConst, remainderConst);
+	partexprs_item = list_head(key->partexprs);
+
+	/* Add an argument for each key column. */
+	for (i = 0; i < key->partnatts; i++)
+	{
+		Node	   *keyCol;
+
+		/* Left operand */
+		if (key->partattrs[i] != 0)
+		{
+			keyCol = (Node *) makeVar(1,
+									  key->partattrs[i],
+									  key->parttypid[i],
+									  key->parttypmod[i],
+									  key->parttypcoll[i],
+									  0);
+		}
+		else
+		{
+			keyCol = (Node *) copyObject(lfirst(partexprs_item));
+			partexprs_item = lnext(partexprs_item);
+		}
+
+		args = lappend(args, keyCol);
+	}
+
+	fexpr = makeFuncExpr(F_SATISFIES_HASH_PARTITION,
+						 BOOLOID,
+						 args,
+						 InvalidOid,
+						 InvalidOid,
+						 COERCE_EXPLICIT_CALL);
+
+	return list_make1(fexpr);
+}
+
+/*
  * get_qual_for_list
  *
  * Returns a list of expressions to use as a list partition's constraint.
@@ -1736,29 +2021,58 @@ get_partition_for_tuple(PartitionDispatch *pd,
 			cur_index = partdesc->boundinfo->null_index;
 		else
 		{
-			/* Else bsearch in partdesc->boundinfo */
-			bool		equal = false;
-
-			cur_offset = partition_bound_bsearch(key, partdesc->boundinfo,
-												 values, false, &equal);
 			switch (key->strategy)
 			{
+				case PARTITION_STRATEGY_HASH:
+					{
+						PartitionBoundInfo boundinfo = partdesc->boundinfo;
+						int		ndatums = boundinfo->ndatums;
+						Datum	datum = boundinfo->datums[ndatums - 1][0];
+						int		modulus = DatumGetInt32(datum);
+						uint32	rowHash = cal_hash_value(key->partsupfunc,
+														 key->partnatts,
+														 values, isnull);
+
+						cur_index = boundinfo->indexes[rowHash % modulus];
+						break;
+					}
+
 				case PARTITION_STRATEGY_LIST:
-					if (cur_offset >= 0 && equal)
-						cur_index = partdesc->boundinfo->indexes[cur_offset];
-					else
-						cur_index = -1;
-					break;
+					{
+						/* bsearch in partdesc->boundinfo */
+						bool		equal = false;
+
+						cur_offset = partition_bound_bsearch(key,
+															 partdesc->boundinfo,
+															 values,
+															 false,
+															 &equal);
+						if (cur_offset >= 0 && equal)
+							cur_index = partdesc->boundinfo->indexes[cur_offset];
+						else
+							cur_index = -1;
+						break;
+					}
 
 				case PARTITION_STRATEGY_RANGE:
+					{
+						bool		equal = false;
 
-					/*
-					 * Offset returned is such that the bound at offset is
-					 * found to be less or equal with the tuple. So, the bound
-					 * at offset+1 would be the upper bound.
-					 */
-					cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
-					break;
+						/* bsearch in partdesc->boundinfo */
+						cur_offset = partition_bound_bsearch(key,
+															 partdesc->boundinfo,
+															 values,
+															 false,
+															 &equal);
+
+						/*
+						 * Offset returned is such that the bound at offset is
+						 * found to be less or equal with the tuple. So, the bound
+						 * at offset+1 would be the upper bound.
+						 */
+						cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
+						break;
+					}
 
 				default:
 					elog(ERROR, "unexpected partition strategy: %d",
@@ -1791,6 +2105,27 @@ get_partition_for_tuple(PartitionDispatch *pd,
 }
 
 /*
+ * Used when sorting hash bounds across all hash modulus
+ * for hash partitioning
+ */
+static int32
+qsort_partition_hbound_cmp(const void *a, const void *b)
+{
+	PartitionHashBound *h1 = (*(PartitionHashBound *const *) a);
+	PartitionHashBound *h2 = (*(PartitionHashBound *const *) b);
+	int v1 = h1->modulus;
+	int v2 = h2->modulus;
+
+	if (v1 < v2)
+		return -1;
+	if (v1 > v2)
+		return 1;
+	if (v1 == v2 && h1->remainder != h2->remainder)
+		return (h1->remainder > h2->remainder) ? 1 : -1;
+	return 0;
+}
+
+/*
  * qsort_partition_list_value_cmp
  *
  * Compare two list partition bound datums
@@ -1967,6 +2302,25 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	switch (key->strategy)
 	{
+		case PARTITION_STRATEGY_HASH:
+			{
+				PartitionBoundSpec *spec = (PartitionBoundSpec *) probe;
+				int	mod = DatumGetInt32(bound_datums[0]);
+
+				if (mod < spec->modulus)
+					cmpval = -1;
+				else if (mod > spec->modulus)
+					cmpval = 1;
+				else if (mod == spec->modulus)
+				{
+					int rem = DatumGetInt32(bound_datums[1]);
+
+					cmpval = rem == spec->remainder ? 0 :
+						(rem < spec->remainder? -1 : 1);
+				}
+
+				break;
+			}
 		case PARTITION_STRATEGY_LIST:
 			cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
 													 key->partcollation[0],
@@ -2050,3 +2404,110 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+/*
+ * Compute the hash value for given partition column values.
+ */
+static uint32
+cal_hash_value(FmgrInfo *partsupfunc, int nkeys, Datum *values, bool *isnull)
+{
+	int		i;
+	uint32	rowHash = 0;
+
+	for (i = 0; i < nkeys; i++)
+	{
+		/* rotate hash left 1 bit before mixing in the next column */
+		rowHash = (rowHash << 1) | ((rowHash & 0x80000000) ? 1 : 0);
+
+		if (!isnull[i])
+		{
+			Datum       colHash;
+
+			Assert(OidIsValid(partsupfunc[i].fn_oid));
+
+			colHash = FunctionCall1(&partsupfunc[i], values[i]);
+			rowHash ^= DatumGetUInt32(colHash);
+		}
+	}
+
+	return rowHash;
+}
+
+/*
+ * satisfies_hash_partition
+ *
+ * This is a SQL-callable function for use in hash partition constraints;
+ * see get_qual_for_hash() for usage.
+ */
+Datum
+satisfies_hash_partition(PG_FUNCTION_ARGS)
+{
+	typedef struct ColumnsHashData
+	{
+		int			n;			/* allocated length of typentry[] */
+		TypeCacheEntry *typentry[PARTITION_MAX_KEYS];
+	}			ColumnsHashData;
+	int			modulus = PG_GETARG_INT32(0);
+	int			remainder = PG_GETARG_INT32(1);
+	short		nkeys = PG_NARGS() - 2;
+	int			i;
+	Datum       values[PARTITION_MAX_KEYS];
+	bool        isnull[PARTITION_MAX_KEYS];
+	FmgrInfo	partsupfunc[PARTITION_MAX_KEYS];
+	ColumnsHashData *my_extra;
+	uint32		rowHash = 0;
+
+	/*
+	 * Cache hash function information, similar to how record_eq() caches
+	 * equality operator information.  (Perhaps no SQL syntax could cause
+	 * PG_NARGS()/nkeys to change between calls through the same FmgrInfo.
+	 * Checking nkeys here is just defensiveness.)
+	 */
+	my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+	if (my_extra == NULL || my_extra->n != nkeys)
+	{
+		fcinfo->flinfo->fn_extra =
+			MemoryContextAllocZero(fcinfo->flinfo->fn_mcxt,
+								   offsetof(ColumnsHashData, typentry) +
+								   sizeof(TypeCacheEntry *) * nkeys);
+		my_extra = (ColumnsHashData *) fcinfo->flinfo->fn_extra;
+		my_extra->n = nkeys;
+	}
+
+	/* Get TypeCacheEntry for each partition column. */
+	for (i = 0; i < nkeys; i++)
+	{
+		/* keys start from third argument of function. */
+		if (!PG_ARGISNULL(i + 2))
+		{
+			Oid			valtype;
+
+			valtype = get_fn_expr_argtype(fcinfo->flinfo, (i + 2));
+			if (!OidIsValid(valtype))
+				elog(ERROR, "could not determine data type of satisfies_hash_partition() input");
+
+			/* Get the hash function. */
+			if (my_extra->typentry[i] == NULL ||
+				my_extra->typentry[i]->type_id != valtype)
+			{
+				my_extra->typentry[i] =
+					lookup_type_cache(valtype, TYPECACHE_HASH_PROC_FINFO);
+				if (!OidIsValid(my_extra->typentry[i]->hash_proc_finfo.fn_oid))
+					ereport(ERROR,
+							(errcode(ERRCODE_UNDEFINED_FUNCTION),
+							 errmsg("could not identify a hash function for type %s",
+									format_type_be(valtype))));
+			}
+
+			values[i] = PG_GETARG_DATUM(i + 2);
+			isnull[i] = false;
+			partsupfunc[i] = my_extra->typentry[i]->hash_proc_finfo;
+		}
+		else
+			isnull[i] = true;
+	}
+
+	rowHash = cal_hash_value(partsupfunc, nkeys, values, isnull);
+
+	PG_RETURN_BOOL(rowHash % modulus == remainder);
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 3cea220..8f6aa96 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -451,7 +451,7 @@ static void RangeVarCallbackForAlterRelation(const RangeVar *rv, Oid relid,
 static bool is_partition_attr(Relation rel, AttrNumber attnum, bool *used_in_expr);
 static PartitionSpec *transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy);
 static void ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
-					  List **partexprs, Oid *partopclass, Oid *partcollation);
+					  List **partexprs, Oid *partopclass, Oid *partcollation, char strategy);
 static void CreateInheritance(Relation child_rel, Relation parent_rel);
 static void RemoveInheritance(Relation child_rel, Relation parent_rel);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
@@ -812,7 +812,7 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 												&strategy);
 		ComputePartitionAttrs(rel, stmt->partspec->partParams,
 							  partattrs, &partexprs, partopclass,
-							  partcollation);
+							  partcollation, strategy);
 
 		partnatts = list_length(stmt->partspec->partParams);
 		StorePartitionKey(rel, strategy, partnatts, partattrs, partexprs,
@@ -12789,6 +12789,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
 		*strategy = PARTITION_STRATEGY_LIST;
 	else if (!pg_strcasecmp(partspec->strategy, "range"))
 		*strategy = PARTITION_STRATEGY_RANGE;
+	else if (!pg_strcasecmp(partspec->strategy, "hash"))
+		*strategy = PARTITION_STRATEGY_HASH;
 	else
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -12844,7 +12846,8 @@ transformPartitionSpec(Relation rel, PartitionSpec *partspec, char *strategy)
  */
 static void
 ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
-					  List **partexprs, Oid *partopclass, Oid *partcollation)
+					  List **partexprs, Oid *partopclass, Oid *partcollation,
+					  char strategy)
 {
 	int			attn;
 	ListCell   *lc;
@@ -12989,27 +12992,49 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
 
 		partcollation[attn] = attcollation;
 
-		/*
-		 * Identify a btree opclass to use. Currently, we use only btree
-		 * operators, which seems enough for list and range partitioning.
-		 */
-		if (!pelem->opclass)
+		if (strategy == PARTITION_STRATEGY_HASH)
 		{
-			partopclass[attn] = GetDefaultOpClass(atttype, BTREE_AM_OID);
+			/* Identify a hash opclass to use */
+			if (!pelem->opclass)
+			{
+				partopclass[attn] = GetDefaultOpClass(atttype, HASH_AM_OID);
 
-			if (!OidIsValid(partopclass[attn]))
-				ereport(ERROR,
-						(errcode(ERRCODE_UNDEFINED_OBJECT),
-				   errmsg("data type %s has no default btree operator class",
-						  format_type_be(atttype)),
-						 errhint("You must specify a btree operator class or define a default btree operator class for the data type.")));
+				if (!OidIsValid(partopclass[attn]))
+					ereport(ERROR,
+							(errcode(ERRCODE_UNDEFINED_OBJECT),
+							 errmsg("data type %s has no default hash operator class",
+									format_type_be(atttype)),
+							 errhint("You must specify a hash operator class or define a default hash operator class for the data type.")));
+			}
+			else
+				partopclass[attn] = ResolveOpClass(pelem->opclass,
+												   atttype,
+												   "hash",
+												   HASH_AM_OID);
 		}
 		else
-			partopclass[attn] = ResolveOpClass(pelem->opclass,
-											   atttype,
-											   "btree",
-											   BTREE_AM_OID);
+		{
+			/*
+			 * Identify a btree opclass to use. Currently, we use only btree
+			 * operators, which seems enough for list and range partitioning.
+			 */
+			if (!pelem->opclass)
+			{
+				partopclass[attn] = GetDefaultOpClass(atttype, BTREE_AM_OID);
 
+				if (!OidIsValid(partopclass[attn]))
+					ereport(ERROR,
+							(errcode(ERRCODE_UNDEFINED_OBJECT),
+							 errmsg("data type %s has no default btree operator class",
+									format_type_be(atttype)),
+							 errhint("You must specify a btree operator class or define a default btree operator class for the data type.")));
+			}
+			else
+				partopclass[attn] = ResolveOpClass(pelem->opclass,
+												   atttype,
+												   "btree",
+												   BTREE_AM_OID);
+		}
 		attn++;
 	}
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 05d8538..250f6ab 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4257,6 +4257,8 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
 	PartitionBoundSpec *newnode = makeNode(PartitionBoundSpec);
 
 	COPY_SCALAR_FIELD(strategy);
+	COPY_SCALAR_FIELD(modulus);
+	COPY_SCALAR_FIELD(remainder);
 	COPY_NODE_FIELD(listdatums);
 	COPY_NODE_FIELD(lowerdatums);
 	COPY_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index d595cd7..3f4dd0f 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2746,6 +2746,8 @@ static bool
 _equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *b)
 {
 	COMPARE_SCALAR_FIELD(strategy);
+	COMPARE_SCALAR_FIELD(modulus);
+	COMPARE_SCALAR_FIELD(remainder);
 	COMPARE_NODE_FIELD(listdatums);
 	COMPARE_NODE_FIELD(lowerdatums);
 	COMPARE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..2a39bb1 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3339,6 +3339,8 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
 	WRITE_NODE_TYPE("PARTITIONBOUND");
 
 	WRITE_CHAR_FIELD(strategy);
+	WRITE_INT_FIELD(modulus);
+	WRITE_INT_FIELD(remainder);
 	WRITE_NODE_FIELD(listdatums);
 	WRITE_NODE_FIELD(lowerdatums);
 	WRITE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 05bf2e9..d1aea69 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2293,6 +2293,8 @@ _readPartitionBoundSpec(void)
 	READ_LOCALS(PartitionBoundSpec);
 
 	READ_CHAR_FIELD(strategy);
+	READ_INT_FIELD(modulus);
+	READ_INT_FIELD(remainder);
 	READ_NODE_FIELD(listdatums);
 	READ_NODE_FIELD(lowerdatums);
 	READ_NODE_FIELD(upperdatums);
diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y
index e833b2e..cbac521 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -573,7 +573,8 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 %type <node>		partbound_datum
 %type <list>		partbound_datum_list
 %type <partrange_datum>	PartitionRangeDatum
-%type <list>		range_datum_list
+%type <list>		range_datum_list hash_partbound
+%type <defelt>		hash_partbound_elem
 
 /*
  * Non-keyword token types.  These are hard-wired into the "flex" lexer.
@@ -2563,8 +2564,35 @@ reloption_elem:
 		;
 
 ForValues:
+			/* a HASH partition*/
+			WITH '(' hash_partbound ')' /*TODO: syntax is not finalised*/
+				{
+					ListCell   *lc;
+					PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+					n->strategy = PARTITION_STRATEGY_HASH;
+
+					foreach (lc, $3)
+					{
+						DefElem    *opt = (DefElem *) lfirst(lc);
+
+						if (strcmp(opt->defname, "modulus") == 0)
+							n->modulus = defGetInt32(opt);
+						else if (strcmp(opt->defname, "remainder") == 0)
+							n->remainder = defGetInt32(opt);
+						else
+							ereport(ERROR,
+									(errcode(ERRCODE_SYNTAX_ERROR),
+									 errmsg("unrecognized hash partition bound specification \"%s\"",
+											opt->defname),
+									 parser_errposition(opt->location)));
+					}
+
+					n->location = @1;
+
+					$$ = (Node *) n;
+				}
 			/* a LIST partition */
-			FOR VALUES IN_P '(' partbound_datum_list ')'
+			| FOR VALUES IN_P '(' partbound_datum_list ')'
 				{
 					PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
 
@@ -2589,6 +2617,20 @@ ForValues:
 				}
 		;
 
+hash_partbound_elem:
+		NonReservedWord Iconst
+			{
+				$$ = makeDefElem($1, (Node *)makeInteger($2), @1);
+			}
+		;
+
+hash_partbound:
+		hash_partbound_elem ',' hash_partbound_elem
+			{
+				$$ = list_make2($1, $3);
+			}
+		;
+
 partbound_datum:
 			Sconst			{ $$ = makeStringConst($1, @1); }
 			| NumericOnly	{ $$ = makeAConst($1, @1); }
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index ff2bab6..de6c0f7 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -3066,7 +3066,33 @@ transformPartitionBound(ParseState *pstate, Relation parent, Node *bound)
 
 	result_spec = copyObject(spec);
 
-	if (strategy == PARTITION_STRATEGY_LIST)
+	if (strategy == PARTITION_STRATEGY_HASH)
+	{
+		if (spec->strategy != PARTITION_STRATEGY_HASH)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("invalid bound specification for a hash partition"),
+					 parser_errposition(pstate, exprLocation(bound))));
+
+		if (spec->modulus <= 0)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("invalid bound specification for a hash partition"),
+					 errhint("modulus must be greater than zero")));
+
+		if (spec->remainder < 0)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("invalid bound specification for a hash partition"),
+					 errhint("remainder must be greater than or equal to zero")));
+
+		if (spec->remainder >= spec->modulus)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("invalid bound specification for a hash partition"),
+					 errhint("modulus must be greater than remainder")));
+	}
+	else if (strategy == PARTITION_STRATEGY_LIST)
 	{
 		ListCell   *cell;
 		char	   *colname;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index b27b77d..f85593c 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -1507,6 +1507,9 @@ pg_get_partkeydef_worker(Oid relid, int prettyFlags)
 
 	switch (form->partstrat)
 	{
+		case PARTITION_STRATEGY_HASH:
+			appendStringInfo(&buf, "HASH");
+			break;
 		case PARTITION_STRATEGY_LIST:
 			appendStringInfo(&buf, "LIST");
 			break;
@@ -8454,6 +8457,13 @@ get_rule_expr(Node *node, deparse_context *context,
 
 				switch (spec->strategy)
 				{
+					case PARTITION_STRATEGY_HASH:
+						Assert(spec->modulus > 0 && spec->remainder >= 0);
+
+						appendStringInfo(buf, "WITH (modulus %d, remainder %d)",
+										 spec->modulus, spec->remainder);
+						break;
+
 					case PARTITION_STRATEGY_LIST:
 						Assert(spec->listdatums != NIL);
 
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 4b9c6e7..a0cea73 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -5361,6 +5361,9 @@ DESCR("pg_controldata init state information as a function");
 DATA(insert OID = 3445 ( pg_import_system_collations PGNSP PGUID 12 100 0 0 0 f f f f t f v r 2 0 2278 "16 4089" _null_ _null_ "{if_not_exists,schema}" _null_ _null_ pg_import_system_collations _null_ _null_ _null_ ));
 DESCR("import collations from operating system");
 
+DATA(insert OID = 5028 ( satisfies_hash_partition PGNSP PGUID 12 1 0 2276 0 f f f f f f i s 3 0 16 "23 23 2276" _null_ _null_ _null_ _null_ _null_ satisfies_hash_partition _null_ _null_ _null_ ));
+DESCR("hash partition CHECK constraint");
+
 /*
  * Symbolic values for provolatile column: these indicate whether the result
  * of a function is dependent *only* on the values of its explicit arguments,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 5afc3eb..1bf00ec 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -728,11 +728,13 @@ typedef struct PartitionElem
 typedef struct PartitionSpec
 {
 	NodeTag		type;
-	char	   *strategy;		/* partitioning strategy ('list' or 'range') */
+	char	   *strategy;		/* partitioning strategy
+								   ('hash', 'list' or 'range') */
 	List	   *partParams;		/* List of PartitionElems */
 	int			location;		/* token location, or -1 if unknown */
 } PartitionSpec;
 
+#define PARTITION_STRATEGY_HASH		'h'
 #define PARTITION_STRATEGY_LIST		'l'
 #define PARTITION_STRATEGY_RANGE	'r'
 
@@ -745,6 +747,10 @@ typedef struct PartitionBoundSpec
 
 	char		strategy;
 
+	/* Hash partition specs */
+	int 		modulus;
+	int			remainder;
+
 	/* List partition values */
 	List	   *listdatums;
 
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 9885fcb..c77563f 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3178,6 +3178,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
 CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
 ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
 ERROR:  partition "fail_part" would overlap partition "part_1"
+DROP TABLE fail_part;
 -- check validation when attaching list partitions
 CREATE TABLE list_parted2 (
 	a int,
@@ -3256,6 +3257,186 @@ DETAIL:  "part_5" is already a child of "list_parted2".
 ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
 ERROR:  circular inheritance not allowed
 DETAIL:  "list_parted2" is already a child of "list_parted2".
+-- check validation when attaching hash partitions
+-- check that target table is partitioned
+CREATE TABLE unparted (
+	a int
+);
+CREATE TABLE fail_part (like unparted);
+ALTER TABLE unparted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  "unparted" is not partitioned
+DROP TABLE unparted, fail_part;
+-- check that partition bound is compatible
+CREATE TABLE hash_parted (
+	a int NOT NULL,
+	b char(2) COLLATE "C",
+	CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE fail_part (LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES FROM (1) TO (10);
+ERROR:  invalid bound specification for a hash partition
+LINE 1: ...hash_parted ATTACH PARTITION fail_part FOR VALUES FROM (1) T...
+                                                             ^
+DROP TABLE fail_part;
+-- check that the table being attached exists
+ALTER TABLE hash_parted ATTACH PARTITION nonexistant WITH (modulus 4, remainder 0);
+ERROR:  relation "nonexistant" does not exist
+-- check ownership of the source table
+CREATE ROLE regress_test_me;
+CREATE ROLE regress_test_not_me;
+CREATE TABLE not_owned_by_me (LIKE hash_parted);
+ALTER TABLE not_owned_by_me OWNER TO regress_test_not_me;
+SET SESSION AUTHORIZATION regress_test_me;
+CREATE TABLE owned_by_me (
+	a int
+) PARTITION BY HASH (a);
+ALTER TABLE owned_by_me ATTACH PARTITION not_owned_by_me WITH (modulus 4, remainder 0);
+ERROR:  must be owner of relation not_owned_by_me
+RESET SESSION AUTHORIZATION;
+DROP TABLE owned_by_me, not_owned_by_me;
+DROP ROLE regress_test_not_me;
+DROP ROLE regress_test_me;
+-- check that the table being attached is not part of regular inheritance
+CREATE TABLE parent (LIKE hash_parted);
+CREATE TABLE child () INHERITS (parent);
+ALTER TABLE hash_parted ATTACH PARTITION child WITH (modulus 4, remainder 1);
+ERROR:  cannot attach inheritance child as partition
+ALTER TABLE hash_parted ATTACH PARTITION parent WITH (modulus 4, remainder 2);
+ERROR:  cannot attach inheritance parent as partition
+DROP TABLE parent CASCADE;
+NOTICE:  drop cascades to table child
+-- check any TEMP-ness
+CREATE TEMP TABLE temp_parted (a int) PARTITION BY HASH (a);
+CREATE TABLE perm_part (a int);
+ALTER TABLE temp_parted ATTACH PARTITION perm_part WITH (modulus 8, remainder 0);
+ERROR:  cannot attach a permanent relation as partition of temporary relation "temp_parted"
+DROP TABLE temp_parted, perm_part;
+-- check that the table being attached is not a typed table
+CREATE TYPE mytype AS (a int);
+CREATE TABLE fail_part OF mytype;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 8, remainder 1);
+ERROR:  cannot attach a typed table as partition
+DROP TYPE mytype CASCADE;
+NOTICE:  drop cascades to table fail_part
+-- check existence (or non-existence) of oid column
+ALTER TABLE hash_parted SET WITH OIDS;
+CREATE TABLE fail_part (a int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  cannot attach table "fail_part" without OIDs as partition of table "hash_parted" with OIDs
+ALTER TABLE hash_parted SET WITHOUT OIDS;
+ALTER TABLE fail_part SET WITH OIDS;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  cannot attach table "fail_part" with OIDs as partition of table "hash_parted" without OIDs
+DROP TABLE fail_part;
+-- check that the table being attached has only columns present in the parent
+CREATE TABLE fail_part (like hash_parted, c int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  table "fail_part" contains column "c" not found in parent "hash_parted"
+DETAIL:  New partition should contain only the columns present in parent.
+DROP TABLE fail_part;
+-- check that the table being attached has every column of the parent
+CREATE TABLE fail_part (a int NOT NULL);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  child table is missing column "b"
+DROP TABLE fail_part;
+-- check that columns match in type, collation and NOT NULL status
+CREATE TABLE fail_part (
+	b char(3),
+	a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  child table "fail_part" has different type for column "b"
+ALTER TABLE fail_part ALTER b TYPE char (2) COLLATE "POSIX";
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  child table "fail_part" has different collation for column "b"
+DROP TABLE fail_part;
+-- check that the table being attached has all constraints of the parent
+CREATE TABLE fail_part (
+	b char(2) COLLATE "C",
+	a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  child table is missing constraint "hcheck_a"
+-- check that the constraint matches in definition with parent's constraint
+ALTER TABLE fail_part ADD CONSTRAINT hcheck_a CHECK (a >= 0);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  child table "fail_part" has different definition for check constraint "hcheck_a"
+DROP TABLE fail_part;
+-- check the attributes and constraints after partition is attached
+CREATE TABLE hpart_1 (
+	a int NOT NULL,
+	b char(2) COLLATE "C",
+	CONSTRAINT hcheck_a CHECK (a > 0)
+);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_1 WITH (modulus 4, remainder 0);
+-- attislocal and conislocal are always false for merged attributes and constraints respectively.
+SELECT attislocal, attinhcount FROM pg_attribute WHERE attrelid = 'hpart_1'::regclass AND attnum > 0;
+ attislocal | attinhcount 
+------------+-------------
+ f          |           1
+ f          |           1
+(2 rows)
+
+SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'hpart_1'::regclass AND conname = 'hcheck_a';
+ conislocal | coninhcount 
+------------+-------------
+ f          |           1
+(1 row)
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ERROR:  partition "fail_part" would overlap partition "hpart_1"
+DROP TABLE fail_part;
+-- check validation when attaching list partitions
+CREATE TABLE hash_parted2 (
+	a int,
+	b char
+) PARTITION BY HASH (a);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+ERROR:  partition constraint is violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+	LIKE hash_parted2
+) PARTITION BY LIST (b);
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+ERROR:  partition constraint is violated by some row
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+-- check that the table being attached is not already a partition
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+ERROR:  "hpart_2" is already a partition
+-- check that circular inheritance is not allowed
+ALTER TABLE hpart_5 ATTACH PARTITION hash_parted2 FOR VALUES IN ('b');
+ERROR:  circular inheritance not allowed
+DETAIL:  "hpart_5" is already a child of "hash_parted2".
+ALTER TABLE hash_parted2 ATTACH PARTITION hash_parted2 WITH (modulus 4, remainder 2);
+ERROR:  circular inheritance not allowed
+DETAIL:  "hash_parted2" is already a child of "hash_parted2".
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 0, remainder 1);
+ERROR:  invalid bound specification for a hash partition
+HINT:  modulus must be greater than zero
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 8, remainder 8);
+ERROR:  invalid bound specification for a hash partition
+HINT:  modulus must be greater than remainder
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 3, remainder 2);
+ERROR:  invalid bound specification for a hash partition
+HINT:  every modulus must be factor of next largest modulus
+DROP TABLE fail_part;
 --
 -- DETACH PARTITION
 --
@@ -3267,12 +3448,19 @@ DROP TABLE regular_table;
 -- check that the partition being detached exists at all
 ALTER TABLE list_parted2 DETACH PARTITION part_4;
 ERROR:  relation "part_4" does not exist
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_4;
+ERROR:  relation "hpart_4" does not exist
 -- check that the partition being detached is actually a partition of the parent
 CREATE TABLE not_a_part (a int);
 ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
 ERROR:  relation "not_a_part" is not a partition of relation "list_parted2"
 ALTER TABLE list_parted2 DETACH PARTITION part_1;
 ERROR:  relation "part_1" is not a partition of relation "list_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION not_a_part;
+ERROR:  relation "not_a_part" is not a partition of relation "hash_parted2"
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+ERROR:  relation "hpart_1" is not a partition of relation "hash_parted2"
+DROP TABLE not_a_part;
 -- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
 -- attislocal/conislocal is set to true
 ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -3290,12 +3478,32 @@ SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::r
 (1 row)
 
 DROP TABLE part_3_4;
+CREATE TABLE hpart_3 PARTITION OF hash_parted2(CONSTRAINT hcheck_a CHECK (a IN (3))) WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_3;
+SELECT attinhcount, attislocal FROM pg_attribute WHERE attrelid = 'hpart_3'::regclass AND attnum > 0;
+ attinhcount | attislocal 
+-------------+------------
+           0 | t
+           0 | t
+(2 rows)
+
+SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'hpart_3'::regclass AND conname = 'hcheck_a';
+ coninhcount | conislocal 
+-------------+------------
+           0 | t
+(1 row)
+
+DROP TABLE hpart_3;
 -- Check ALTER TABLE commands for partitioned tables and partitions
 -- cannot add/drop column to/from *only* the parent
 ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
 ERROR:  column must be added to child tables too
 ALTER TABLE ONLY list_parted2 DROP COLUMN b;
 ERROR:  column must be dropped from child tables too
+ALTER TABLE ONLY hash_parted2 ADD COLUMN c int;
+ERROR:  column must be added to child tables too
+ALTER TABLE ONLY hash_parted2 DROP COLUMN b;
+ERROR:  column must be dropped from child tables too
 -- cannot add a column to partition or drop an inherited one
 ALTER TABLE part_2 ADD COLUMN c text;
 ERROR:  cannot add column to a partition
@@ -3313,17 +3521,32 @@ ALTER TABLE ONLY list_parted2 add constraint check_b check (b <> 'zz');
 ERROR:  constraint must be added to child tables too
 ALTER TABLE list_parted2 add constraint check_b check (b <> 'zz') NO INHERIT;
 ERROR:  cannot add NO INHERIT constraint to partitioned table "list_parted2"
+ALTER TABLE ONLY hash_parted2 ALTER b SET NOT NULL;
+ERROR:  constraint must be added to child tables too
+ALTER TABLE ONLY hash_parted2 add constraint hcheck_b check (b <> 'zz');
+ERROR:  constraint must be added to child tables too
+ALTER TABLE hash_parted2 add constraint hcheck_b check (b <> 'zz') NO INHERIT;
+ERROR:  cannot add NO INHERIT constraint to partitioned table "hash_parted2"
 -- cannot drop inherited NOT NULL or check constraints from partition
 ALTER TABLE list_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT check_a2 CHECK (a > 0);
 ALTER TABLE part_2 ALTER b DROP NOT NULL;
 ERROR:  column "b" is marked NOT NULL in parent table
 ALTER TABLE part_2 DROP CONSTRAINT check_a2;
 ERROR:  cannot drop inherited constraint "check_a2" of relation "part_2"
+ALTER TABLE hash_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT hcheck_a2 CHECK (a > 0);
+ALTER TABLE hpart_2 ALTER b DROP NOT NULL;
+ERROR:  column "b" is marked NOT NULL in parent table
+ALTER TABLE hpart_2 DROP CONSTRAINT hcheck_a2;
+ERROR:  cannot drop inherited constraint "hcheck_a2" of relation "hpart_2"
 -- cannot drop NOT NULL or check constraints from *only* the parent
 ALTER TABLE ONLY list_parted2 ALTER a DROP NOT NULL;
 ERROR:  constraint must be dropped from child tables too
 ALTER TABLE ONLY list_parted2 DROP CONSTRAINT check_a2;
 ERROR:  constraint must be dropped from child tables too
+ALTER TABLE ONLY hash_parted2 ALTER a DROP NOT NULL;
+ERROR:  constraint must be dropped from child tables too
+ALTER TABLE ONLY hash_parted2 DROP CONSTRAINT hcheck_a2;
+ERROR:  constraint must be dropped from child tables too
 -- check that a partition cannot participate in regular inheritance
 CREATE TABLE inh_test () INHERITS (part_2);
 ERROR:  cannot inherit from partition "part_2"
@@ -3339,10 +3562,18 @@ ALTER TABLE list_parted2 DROP COLUMN b;
 ERROR:  cannot drop column named in partition key
 ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 ERROR:  cannot alter type of column named in partition key
+-- similarly, hpart_5 which is hash_parted2's partition,
+-- is partitioned on b;
+ALTER TABLE hash_parted2 DROP COLUMN b;
+ERROR:  cannot drop column named in partition key
+ALTER TABLE hash_parted2 ALTER COLUMN b TYPE text;
+ERROR:  cannot alter type of column named in partition key
 -- cleanup: avoid using CASCADE
 DROP TABLE list_parted, part_1;
 DROP TABLE list_parted2, part_2, part_5, part_5_a;
 DROP TABLE range_parted, part1, part2;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
 -- more tests for certain multi-level partitioning scenarios
 create table p (a int, b int) partition by range (a, b);
 create table p1 (b int, a int not null) partition by range (b);
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 20eb3d3..4ccb524 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -340,11 +340,6 @@ CREATE TABLE partitioned (
 ) PARTITION BY RANGE (const_func());
 ERROR:  cannot use constant expression as partition key
 DROP FUNCTION const_func();
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
-	a int
-) PARTITION BY HASH (a);
-ERROR:  unrecognized partitioning strategy "hash"
 -- specified column must be present in the table
 CREATE TABLE partitioned (
 	a int
@@ -505,6 +500,21 @@ ERROR:  TO must specify exactly one value per partitioning column
 -- cannot specify null values in range bounds
 CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (unbounded);
 ERROR:  cannot specify NULL in range bound
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+	a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted WITH (modulus 25, remainder 2);
+ERROR:  invalid bound specification for a hash partition
+HINT:  every modulus must be factor of next largest modulus
+-- trying to specify range for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+ERROR:  invalid bound specification for a hash partition
+LINE 1: ...BLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a',...
+                                                             ^
 -- check if compatible with the specified parent
 -- cannot create as partition of a non-partitioned table
 CREATE TABLE unparted (
@@ -512,6 +522,8 @@ CREATE TABLE unparted (
 );
 CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
 ERROR:  "unparted" is not partitioned
+CREATE TABLE fail_part PARTITION OF unparted WITH (modulus 2, remainder 1);
+ERROR:  "unparted" is not partitioned
 DROP TABLE unparted;
 -- cannot create a permanent rel as partition of a temp rel
 CREATE TEMP TABLE temp_parted (
@@ -519,6 +531,8 @@ CREATE TEMP TABLE temp_parted (
 ) PARTITION BY LIST (a);
 CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
 ERROR:  cannot create a permanent relation as partition of temporary relation "temp_parted"
+CREATE TABLE fail_part PARTITION OF temp_parted WITH (modulus 2, remainder 1);
+ERROR:  cannot create a permanent relation as partition of temporary relation "temp_parted"
 DROP TABLE temp_parted;
 -- cannot create a table with oids as partition of table without oids
 CREATE TABLE no_oids_parted (
@@ -526,6 +540,8 @@ CREATE TABLE no_oids_parted (
 ) PARTITION BY RANGE (a) WITHOUT OIDS;
 CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
 ERROR:  cannot create table with OIDs as partition of table without OIDs
+CREATE TABLE fail_part PARTITION OF no_oids_parted WITH (modulus 2, remainder 1) WITH OIDS;
+ERROR:  cannot create table with OIDs as partition of table without OIDs
 DROP TABLE no_oids_parted;
 -- If the partitioned table has oids, then the partition must have them.
 -- If the WITHOUT OIDS option is specified for partition, it is overridden.
@@ -533,6 +549,10 @@ CREATE TABLE oids_parted (
 	a int
 ) PARTITION BY RANGE (a) WITH OIDS;
 CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted WITH (modulus 2, remainder 1) WITHOUT OIDS;
+ERROR:  invalid bound specification for a range partition
+LINE 1: CREATE TABLE fail_part PARTITION OF oids_parted WITH (modulu...
+                                                        ^
 \d+ part_forced_oids
                              Table "public.part_forced_oids"
  Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description 
@@ -591,6 +611,25 @@ ERROR:  partition "fail_part" would overlap partition "part12"
 -- more specific ranges
 CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
 ERROR:  partition "fail_part" would overlap partition "part10"
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+	a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 2, remainder 1);
+ERROR:  partition "fail_part" would overlap partition "h2part_4"
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 0, remainder 1);
+ERROR:  invalid bound specification for a hash partition
+HINT:  modulus must be greater than zero
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 8, remainder 8);
+ERROR:  invalid bound specification for a hash partition
+HINT:  modulus must be greater than remainder
 -- check schema propagation from parent
 CREATE TABLE parted (
 	a text,
@@ -674,3 +713,5 @@ DROP TABLE range_parted;
 DROP TABLE list_parted2, part_ab, part_null_z;
 DROP TABLE range_parted2, part0, part1, part2, part3;
 DROP TABLE range_parted3, part00, part10, part11, part12;
+DROP TABLE hash_parted, hpart_1, hpart_2;
+DROP TABLE hash_parted2, h2part_1, h2part_2, h2part_3, h2part_4;
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index 81af3ef..48db21d 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -313,11 +313,47 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
  part_null     |    |     1 |     1
 (9 rows)
 
+-- direct partition inserts should check hash partition bound constraint
+create table hash_parted (
+	a text,
+	b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+-- fail
+insert into hpart1 values ('a', 13);
+ERROR:  new row for relation "hpart1" violates partition constraint
+DETAIL:  Failing row contains (a, 13).
+insert into hpart1 values ('b', 3);
+ERROR:  new row for relation "hpart1" violates partition constraint
+DETAIL:  Failing row contains (b, 3).
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart4 values ('b', 21);
+ERROR:  new row for relation "hpart4" violates partition constraint
+DETAIL:  Failing row contains (b, 21).
+insert into hpart4 values ('a', 10);
+ERROR:  new row for relation "hpart4" violates partition constraint
+DETAIL:  Failing row contains (a, 10).
+-- ok
+insert into hpart4 values ('c', 6);
+-- fail
+insert into hpart1 values (null);
+ERROR:  new row for relation "hpart1" violates partition constraint
+DETAIL:  Failing row contains (null, null).
+-- ok
+insert into hpart3 values (null);
+insert into hpart1 values (1);
 -- cleanup
 drop table part1, part2, part3, part4, range_parted;
 drop table part_ee_ff3_1, part_ee_ff3_2, part_ee_ff1, part_ee_ff2, part_ee_ff3;
 drop table part_ee_ff, part_gg2_2, part_gg2_1, part_gg2, part_gg1, part_gg;
 drop table part_aa_bb, part_cc_dd, part_null, list_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
 -- more tests for certain multi-level partitioning scenarios
 create table p (a int, b int) partition by range (a, b);
 create table p1 (b int not null, a int not null) partition by range ((b+0));
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index a1e9255..c41e26e 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,6 +218,26 @@ ERROR:  new row for relation "part_b_10_b_20" violates partition constraint
 DETAIL:  Failing row contains (b, 9).
 -- ok
 update range_parted set b = b + 1 where b = 10;
+create table hash_parted (
+	a text,
+	b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+ERROR:  new row for relation "hpart1" violates partition constraint
+DETAIL:  Failing row contains (c, 1).
+update hash_parted set b = b - 1 where b = 1;
+ERROR:  new row for relation "hpart1" violates partition constraint
+DETAIL:  Failing row contains (b, 0).
+-- ok
+update hash_parted set b = b + 8 where b = 1;
 -- cleanup
 drop table range_parted cascade;
 NOTICE:  drop cascades to 4 other objects
@@ -225,3 +245,4 @@ DETAIL:  drop cascades to table part_a_1_a_10
 drop cascades to table part_a_10_a_20
 drop cascades to table part_b_1_b_10
 drop cascades to table part_b_10_b_20
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index f7b754f..e545102 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2050,6 +2050,7 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
 -- check that the new partition won't overlap with an existing partition
 CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
 ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+DROP TABLE fail_part;
 
 -- check validation when attaching list partitions
 CREATE TABLE list_parted2 (
@@ -2135,6 +2136,166 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 ALTER TABLE part_5 ATTACH PARTITION list_parted2 FOR VALUES IN ('b');
 ALTER TABLE list_parted2 ATTACH PARTITION list_parted2 FOR VALUES IN (0);
 
+-- check validation when attaching hash partitions
+
+-- check that target table is partitioned
+CREATE TABLE unparted (
+	a int
+);
+CREATE TABLE fail_part (like unparted);
+ALTER TABLE unparted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE unparted, fail_part;
+
+-- check that partition bound is compatible
+CREATE TABLE hash_parted (
+	a int NOT NULL,
+	b char(2) COLLATE "C",
+	CONSTRAINT hcheck_a CHECK (a > 0)
+) PARTITION BY HASH (a);
+CREATE TABLE fail_part (LIKE hash_parted);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part FOR VALUES FROM (1) TO (10);
+DROP TABLE fail_part;
+
+-- check that the table being attached exists
+ALTER TABLE hash_parted ATTACH PARTITION nonexistant WITH (modulus 4, remainder 0);
+
+-- check ownership of the source table
+CREATE ROLE regress_test_me;
+CREATE ROLE regress_test_not_me;
+CREATE TABLE not_owned_by_me (LIKE hash_parted);
+ALTER TABLE not_owned_by_me OWNER TO regress_test_not_me;
+SET SESSION AUTHORIZATION regress_test_me;
+CREATE TABLE owned_by_me (
+	a int
+) PARTITION BY HASH (a);
+ALTER TABLE owned_by_me ATTACH PARTITION not_owned_by_me WITH (modulus 4, remainder 0);
+RESET SESSION AUTHORIZATION;
+DROP TABLE owned_by_me, not_owned_by_me;
+DROP ROLE regress_test_not_me;
+DROP ROLE regress_test_me;
+
+-- check that the table being attached is not part of regular inheritance
+CREATE TABLE parent (LIKE hash_parted);
+CREATE TABLE child () INHERITS (parent);
+ALTER TABLE hash_parted ATTACH PARTITION child WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted ATTACH PARTITION parent WITH (modulus 4, remainder 2);
+DROP TABLE parent CASCADE;
+
+-- check any TEMP-ness
+CREATE TEMP TABLE temp_parted (a int) PARTITION BY HASH (a);
+CREATE TABLE perm_part (a int);
+ALTER TABLE temp_parted ATTACH PARTITION perm_part WITH (modulus 8, remainder 0);
+DROP TABLE temp_parted, perm_part;
+
+-- check that the table being attached is not a typed table
+CREATE TYPE mytype AS (a int);
+CREATE TABLE fail_part OF mytype;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 8, remainder 1);
+DROP TYPE mytype CASCADE;
+
+-- check existence (or non-existence) of oid column
+ALTER TABLE hash_parted SET WITH OIDS;
+CREATE TABLE fail_part (a int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+
+ALTER TABLE hash_parted SET WITHOUT OIDS;
+ALTER TABLE fail_part SET WITH OIDS;
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that the table being attached has only columns present in the parent
+CREATE TABLE fail_part (like hash_parted, c int);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that the table being attached has every column of the parent
+CREATE TABLE fail_part (a int NOT NULL);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that columns match in type, collation and NOT NULL status
+CREATE TABLE fail_part (
+	b char(3),
+	a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+ALTER TABLE fail_part ALTER b TYPE char (2) COLLATE "POSIX";
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check that the table being attached has all constraints of the parent
+CREATE TABLE fail_part (
+	b char(2) COLLATE "C",
+	a int NOT NULL
+);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+
+-- check that the constraint matches in definition with parent's constraint
+ALTER TABLE fail_part ADD CONSTRAINT hcheck_a CHECK (a >= 0);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check the attributes and constraints after partition is attached
+CREATE TABLE hpart_1 (
+	a int NOT NULL,
+	b char(2) COLLATE "C",
+	CONSTRAINT hcheck_a CHECK (a > 0)
+);
+ALTER TABLE hash_parted ATTACH PARTITION hpart_1 WITH (modulus 4, remainder 0);
+-- attislocal and conislocal are always false for merged attributes and constraints respectively.
+SELECT attislocal, attinhcount FROM pg_attribute WHERE attrelid = 'hpart_1'::regclass AND attnum > 0;
+SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'hpart_1'::regclass AND conname = 'hcheck_a';
+
+-- check that the new partition won't overlap with an existing partition
+CREATE TABLE fail_part (LIKE hpart_1 INCLUDING CONSTRAINTS);
+ALTER TABLE hash_parted ATTACH PARTITION fail_part WITH (modulus 4, remainder 0);
+DROP TABLE fail_part;
+
+-- check validation when attaching list partitions
+CREATE TABLE hash_parted2 (
+	a int,
+	b char
+) PARTITION BY HASH (a);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_2 (LIKE hash_parted2);
+INSERT INTO hpart_2 VALUES (3, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+
+-- should be ok after deleting the bad row
+DELETE FROM hpart_2;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+
+-- check that leaf partitions are scanned when attaching a partitioned
+-- table
+CREATE TABLE hpart_5 (
+	LIKE hash_parted2
+) PARTITION BY LIST (b);
+
+-- check that violating rows are correctly reported
+CREATE TABLE hpart_5_a PARTITION OF hpart_5 FOR VALUES IN ('a');
+INSERT INTO hpart_5_a (a, b) VALUES (6, 'a');
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+
+-- delete the faulting row and also add a constraint to skip the scan
+DELETE FROM hpart_5_a WHERE a NOT IN (3);
+ALTER TABLE hpart_5 ADD CONSTRAINT hcheck_a CHECK (a IN (5)), ALTER a SET NOT NULL;
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_5 WITH (modulus 4, remainder 2);
+
+-- check that the table being attached is not already a partition
+ALTER TABLE hash_parted2 ATTACH PARTITION hpart_2 WITH (modulus 4, remainder 0);
+
+-- check that circular inheritance is not allowed
+ALTER TABLE hpart_5 ATTACH PARTITION hash_parted2 FOR VALUES IN ('b');
+ALTER TABLE hash_parted2 ATTACH PARTITION hash_parted2 WITH (modulus 4, remainder 2);
+
+-- check that the table being attach is with valid modulus and remainder value
+CREATE TABLE fail_part(LIKE hash_parted2);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 0, remainder 1);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 8, remainder 8);
+ALTER TABLE hash_parted2 ATTACH PARTITION fail_part WITH (modulus 3, remainder 2);
+DROP TABLE fail_part;
+
 --
 -- DETACH PARTITION
 --
@@ -2146,12 +2307,17 @@ DROP TABLE regular_table;
 
 -- check that the partition being detached exists at all
 ALTER TABLE list_parted2 DETACH PARTITION part_4;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_4;
 
 -- check that the partition being detached is actually a partition of the parent
 CREATE TABLE not_a_part (a int);
 ALTER TABLE list_parted2 DETACH PARTITION not_a_part;
 ALTER TABLE list_parted2 DETACH PARTITION part_1;
 
+ALTER TABLE hash_parted2 DETACH PARTITION not_a_part;
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_1;
+DROP TABLE not_a_part;
+
 -- check that, after being detached, attinhcount/coninhcount is dropped to 0 and
 -- attislocal/conislocal is set to true
 ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
@@ -2159,11 +2325,18 @@ SELECT attinhcount, attislocal FROM pg_attribute WHERE attrelid = 'part_3_4'::re
 SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'part_3_4'::regclass AND conname = 'check_a';
 DROP TABLE part_3_4;
 
+CREATE TABLE hpart_3 PARTITION OF hash_parted2(CONSTRAINT hcheck_a CHECK (a IN (3))) WITH (modulus 4, remainder 1);
+ALTER TABLE hash_parted2 DETACH PARTITION hpart_3;
+SELECT attinhcount, attislocal FROM pg_attribute WHERE attrelid = 'hpart_3'::regclass AND attnum > 0;
+SELECT coninhcount, conislocal FROM pg_constraint WHERE conrelid = 'hpart_3'::regclass AND conname = 'hcheck_a';
+DROP TABLE hpart_3;
 -- Check ALTER TABLE commands for partitioned tables and partitions
 
 -- cannot add/drop column to/from *only* the parent
 ALTER TABLE ONLY list_parted2 ADD COLUMN c int;
 ALTER TABLE ONLY list_parted2 DROP COLUMN b;
+ALTER TABLE ONLY hash_parted2 ADD COLUMN c int;
+ALTER TABLE ONLY hash_parted2 DROP COLUMN b;
 
 -- cannot add a column to partition or drop an inherited one
 ALTER TABLE part_2 ADD COLUMN c text;
@@ -2178,15 +2351,26 @@ ALTER TABLE ONLY list_parted2 ALTER b SET NOT NULL;
 ALTER TABLE ONLY list_parted2 add constraint check_b check (b <> 'zz');
 ALTER TABLE list_parted2 add constraint check_b check (b <> 'zz') NO INHERIT;
 
+ALTER TABLE ONLY hash_parted2 ALTER b SET NOT NULL;
+ALTER TABLE ONLY hash_parted2 add constraint hcheck_b check (b <> 'zz');
+ALTER TABLE hash_parted2 add constraint hcheck_b check (b <> 'zz') NO INHERIT;
+
 -- cannot drop inherited NOT NULL or check constraints from partition
 ALTER TABLE list_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT check_a2 CHECK (a > 0);
 ALTER TABLE part_2 ALTER b DROP NOT NULL;
 ALTER TABLE part_2 DROP CONSTRAINT check_a2;
 
+ALTER TABLE hash_parted2 ALTER b SET NOT NULL, ADD CONSTRAINT hcheck_a2 CHECK (a > 0);
+ALTER TABLE hpart_2 ALTER b DROP NOT NULL;
+ALTER TABLE hpart_2 DROP CONSTRAINT hcheck_a2;
+
 -- cannot drop NOT NULL or check constraints from *only* the parent
 ALTER TABLE ONLY list_parted2 ALTER a DROP NOT NULL;
 ALTER TABLE ONLY list_parted2 DROP CONSTRAINT check_a2;
 
+ALTER TABLE ONLY hash_parted2 ALTER a DROP NOT NULL;
+ALTER TABLE ONLY hash_parted2 DROP CONSTRAINT hcheck_a2;
+
 -- check that a partition cannot participate in regular inheritance
 CREATE TABLE inh_test () INHERITS (part_2);
 CREATE TABLE inh_test (LIKE part_2);
@@ -2199,10 +2383,17 @@ ALTER TABLE part_2 INHERIT inh_test;
 ALTER TABLE list_parted2 DROP COLUMN b;
 ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 
+-- similarly, hpart_5 which is hash_parted2's partition,
+-- is partitioned on b;
+ALTER TABLE hash_parted2 DROP COLUMN b;
+ALTER TABLE hash_parted2 ALTER COLUMN b TYPE text;
+
 -- cleanup: avoid using CASCADE
 DROP TABLE list_parted, part_1;
 DROP TABLE list_parted2, part_2, part_5, part_5_a;
 DROP TABLE range_parted, part1, part2;
+DROP TABLE hash_parted, hpart_1;
+DROP TABLE hash_parted2, hpart_2, hpart_5, hpart_5_a;
 
 -- more tests for certain multi-level partitioning scenarios
 create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index f41dd71..268d77d 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -350,11 +350,6 @@ CREATE TABLE partitioned (
 ) PARTITION BY RANGE (const_func());
 DROP FUNCTION const_func();
 
--- only accept "list" and "range" as partitioning strategy
-CREATE TABLE partitioned (
-	a int
-) PARTITION BY HASH (a);
-
 -- specified column must be present in the table
 CREATE TABLE partitioned (
 	a int
@@ -473,6 +468,17 @@ CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM ('a') TO ('z',
 -- cannot specify null values in range bounds
 CREATE TABLE fail_part PARTITION OF range_parted FOR VALUES FROM (null) TO (unbounded);
 
+-- check partition bound syntax for the hash partition
+CREATE TABLE hash_parted (
+	a int
+) PARTITION BY HASH (a);
+CREATE TABLE hpart_1 PARTITION OF hash_parted WITH (modulus 10, remainder 1);
+CREATE TABLE hpart_2 PARTITION OF hash_parted WITH (modulus 50, remainder 0);
+-- modulus 25 is factor of modulus of 50 but 10 is not factor of 25.
+CREATE TABLE fail_part PARTITION OF hash_parted WITH (modulus 25, remainder 2);
+-- trying to specify range for hash partitioned table
+CREATE TABLE fail_part PARTITION OF hash_parted FOR VALUES FROM ('a', 1) TO ('z');
+
 -- check if compatible with the specified parent
 
 -- cannot create as partition of a non-partitioned table
@@ -480,6 +486,7 @@ CREATE TABLE unparted (
 	a int
 );
 CREATE TABLE fail_part PARTITION OF unparted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF unparted WITH (modulus 2, remainder 1);
 DROP TABLE unparted;
 
 -- cannot create a permanent rel as partition of a temp rel
@@ -487,6 +494,7 @@ CREATE TEMP TABLE temp_parted (
 	a int
 ) PARTITION BY LIST (a);
 CREATE TABLE fail_part PARTITION OF temp_parted FOR VALUES IN ('a');
+CREATE TABLE fail_part PARTITION OF temp_parted WITH (modulus 2, remainder 1);
 DROP TABLE temp_parted;
 
 -- cannot create a table with oids as partition of table without oids
@@ -494,6 +502,7 @@ CREATE TABLE no_oids_parted (
 	a int
 ) PARTITION BY RANGE (a) WITHOUT OIDS;
 CREATE TABLE fail_part PARTITION OF no_oids_parted FOR VALUES FROM (1) TO (10) WITH OIDS;
+CREATE TABLE fail_part PARTITION OF no_oids_parted WITH (modulus 2, remainder 1) WITH OIDS;
 DROP TABLE no_oids_parted;
 
 -- If the partitioned table has oids, then the partition must have them.
@@ -502,6 +511,7 @@ CREATE TABLE oids_parted (
 	a int
 ) PARTITION BY RANGE (a) WITH OIDS;
 CREATE TABLE part_forced_oids PARTITION OF oids_parted FOR VALUES FROM (1) TO (10) WITHOUT OIDS;
+CREATE TABLE fail_part PARTITION OF oids_parted WITH (modulus 2, remainder 1) WITHOUT OIDS;
 \d+ part_forced_oids
 DROP TABLE oids_parted, part_forced_oids;
 
@@ -553,6 +563,21 @@ CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1,
 -- more specific ranges
 CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, unbounded) TO (1, unbounded);
 
+-- check for partition bound overlap and other invalid specifications for the hash partition
+CREATE TABLE hash_parted2 (
+	a varchar
+) PARTITION BY HASH (a);
+CREATE TABLE h2part_1 PARTITION OF hash_parted2 WITH (modulus 4, remainder 2);
+CREATE TABLE h2part_2 PARTITION OF hash_parted2 WITH (modulus 8, remainder 0);
+CREATE TABLE h2part_3 PARTITION OF hash_parted2 WITH (modulus 8, remainder 4);
+CREATE TABLE h2part_4 PARTITION OF hash_parted2 WITH (modulus 8, remainder 5);
+-- overlap with part_4
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 2, remainder 1);
+-- modulus must be greater than zero
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 0, remainder 1);
+-- remainder must be greater than or equal to zero and less than modulus
+CREATE TABLE fail_part PARTITION OF hash_parted2 WITH (modulus 8, remainder 8);
+
 -- check schema propagation from parent
 
 CREATE TABLE parted (
@@ -602,3 +627,5 @@ DROP TABLE range_parted;
 DROP TABLE list_parted2, part_ab, part_null_z;
 DROP TABLE range_parted2, part0, part1, part2, part3;
 DROP TABLE range_parted3, part00, part10, part11, part12;
+DROP TABLE hash_parted, hpart_1, hpart_2;
+DROP TABLE hash_parted2, h2part_1, h2part_2, h2part_3, h2part_4;
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index 454e1ce..36ebb45 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -185,11 +185,41 @@ insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a);
 insert into list_parted (b) values (1);
 select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1;
 
+-- direct partition inserts should check hash partition bound constraint
+create table hash_parted (
+	a text,
+	b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+
+-- fail
+insert into hpart1 values ('a', 13);
+insert into hpart1 values ('b', 3);
+-- ok
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+-- fail
+insert into hpart4 values ('b', 21);
+insert into hpart4 values ('a', 10);
+-- ok
+insert into hpart4 values ('c', 6);
+
+-- fail
+insert into hpart1 values (null);
+
+-- ok
+insert into hpart3 values (null);
+insert into hpart1 values (1);
+
 -- cleanup
 drop table part1, part2, part3, part4, range_parted;
 drop table part_ee_ff3_1, part_ee_ff3_2, part_ee_ff1, part_ee_ff2, part_ee_ff3;
 drop table part_ee_ff, part_gg2_2, part_gg2_1, part_gg2, part_gg1, part_gg;
 drop table part_aa_bb, part_cc_dd, part_null, list_parted;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
 
 -- more tests for certain multi-level partitioning scenarios
 create table p (a int, b int) partition by range (a, b);
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index d7721ed..4fc0b21 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,24 @@ update range_parted set b = b - 1 where b = 10;
 -- ok
 update range_parted set b = b + 1 where b = 10;
 
+create table hash_parted (
+	a text,
+	b int
+) partition by hash (a, b);
+create table hpart1 partition of hash_parted with (modulus 2, remainder 1);
+create table hpart2 partition of hash_parted with (modulus 4, remainder 2);
+create table hpart3 partition of hash_parted with (modulus 8, remainder 0);
+create table hpart4 partition of hash_parted with (modulus 8, remainder 4);
+insert into hpart1 values ('b', 1);
+insert into hpart2 values ('c', 1);
+insert into hpart4 values ('c', 6);
+
+-- fail
+update hpart1 set a = 'c' where a = 'b';
+update hash_parted set b = b - 1 where b = 1;
+-- ok
+update hash_parted set b = b + 8 where b = 1;
+
 -- cleanup
 drop table range_parted cascade;
+drop table hash_parted, hpart1, hpart2, hpart3, hpart4;
-- 
2.6.2

