diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 0ce94f3..a009b96 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -1755,9 +1755,11 @@ RemoveAttrDefaultById(Oid attrdefId)
 void
 heap_drop_with_catalog(Oid relid)
 {
-	Relation	rel;
+	Relation	rel,
+				parentRel;
 	HeapTuple	tuple;
-	Oid			parentOid = InvalidOid;
+	Oid			parentOid = InvalidOid,
+				defaultPartOid = InvalidOid;
 
 	/*
 	 * To drop a partition safely, we must grab exclusive lock on its parent,
@@ -1772,7 +1774,15 @@ heap_drop_with_catalog(Oid relid)
 	if (((Form_pg_class) GETSTRUCT(tuple))->relispartition)
 	{
 		parentOid = get_partition_parent(relid);
-		LockRelationOid(parentOid, AccessExclusiveLock);
+		parentRel = heap_open(parentOid, AccessExclusiveLock);
+
+		/*
+		 * Need to take a lock on the default partition, refer comment for
+		 * locking the default partition in DefineRelation().
+		 */
+		defaultPartOid = RelationGetDefaultPartitionOid(parentRel);
+		if (OidIsValid(defaultPartOid))
+			LockRelationOid(defaultPartOid, AccessExclusiveLock);
 	}
 
 	ReleaseSysCache(tuple);
@@ -1883,11 +1893,19 @@ heap_drop_with_catalog(Oid relid)
 	if (OidIsValid(parentOid))
 	{
 		/*
+		 * Invalidate default partition's relcache, refer comment for
+		 * invalidating default partition relcache in StorePartitionBound().
+		 */
+		if (OidIsValid(defaultPartOid))
+			CacheInvalidateRelcacheByRelid(defaultPartOid);
+
+		/*
 		 * Invalidate the parent's relcache so that the partition is no longer
 		 * included in its partition descriptor.
 		 */
 		CacheInvalidateRelcacheByRelid(parentOid);
 		/* keep the lock */
+		heap_close(parentRel, NoLock);
 	}
 }
 
@@ -3215,8 +3233,10 @@ RemovePartitionKeyByRelId(Oid relid)
  *		Update pg_class tuple of rel to store the partition bound and set
  *		relispartition to true
  *
- * Also, invalidate the parent's relcache, so that the next rebuild will load
- * the new partition's info into its partition descriptor.
+ * Also, invalidate the parent's and a sibling default partition's relcache,
+ * so that the next rebuild will load the new partition's info into parent's
+ * partition descriptor and default partition constraints(which are dependent
+ * on other partition bounds) are built anew.
  */
 void
 StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
@@ -3227,6 +3247,7 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	Datum		new_val[Natts_pg_class];
 	bool		new_null[Natts_pg_class],
 				new_repl[Natts_pg_class];
+	Oid			defaultPartOid;
 
 	/* Update pg_class tuple */
 	classRel = heap_open(RelationRelationId, RowExclusiveLock);
@@ -3264,5 +3285,16 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
 	heap_freetuple(newtuple);
 	heap_close(classRel, RowExclusiveLock);
 
+	/*
+	 * The default partition constraints depend upon the partition bounds of
+	 * other partitions. Adding a new(or even removing existing) partition
+	 * would invalidate the default partition constraints. Invalidate the
+	 * default partition's relcache so that the constraints are built anew
+	 * and any plans dependent on those constraints are invalidated as well.
+	 */
+	defaultPartOid = RelationGetDefaultPartitionOid(parent);
+	if (OidIsValid(defaultPartOid))
+		CacheInvalidateRelcacheByRelid(defaultPartOid);
+
 	CacheInvalidateRelcache(parent);
 }
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 5c5a9e1..3276023 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -35,6 +35,7 @@
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/var.h"
 #include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
@@ -88,9 +89,12 @@ typedef struct PartitionBoundInfoData
 								 * partitioned table) */
 	int			null_index;		/* Index of the null-accepting partition; -1
 								 * if there isn't one */
+	int			default_index;	/* Index of the default partition if any; -1
+								 * if there isn't one */
 } PartitionBoundInfoData;
 
 #define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
 
 /*
  * When qsort'ing partition bounds after reading from the catalog, each bound
@@ -121,14 +125,15 @@ static int32 qsort_partition_rbound_cmp(const void *a, const void *b,
 static Oid get_partition_operator(PartitionKey key, int col,
 					   StrategyNumber strategy, bool *need_relabel);
 static Expr *make_partition_op_expr(PartitionKey key, int keynum,
-					   uint16 strategy, Expr *arg1, Expr *arg2);
+					   uint16 strategy, Expr *arg1, Expr *arg2,
+					   bool is_default);
 static void get_range_key_properties(PartitionKey key, int keynum,
 						 PartitionRangeDatum *ldatum,
 						 PartitionRangeDatum *udatum,
 						 ListCell **partexprs_item,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
-static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
+static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
 static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
 static List *generate_partition_qual(Relation rel);
 
@@ -147,6 +152,8 @@ static int32 partition_bound_cmp(PartitionKey key,
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
+static void check_default_allows_bound(Relation parent,
+						   PartitionBoundSpec *new_spec);
 
 /*
  * RelationBuildPartitionDesc
@@ -174,6 +181,7 @@ RelationBuildPartitionDesc(Relation rel)
 	/* List partitioning specific */
 	PartitionListValue **all_values = NULL;
 	int			null_index = -1;
+	int			default_index = -1;
 
 	/* Range partitioning specific */
 	PartitionRangeBound **rbounds = NULL;
@@ -254,6 +262,18 @@ RelationBuildPartitionDesc(Relation rel)
 				if (spec->strategy != PARTITION_STRATEGY_LIST)
 					elog(ERROR, "invalid strategy in partition bound spec");
 
+				/*
+				 * Note the index of the partition bound spec for the default
+				 * partition. There's no datum to add to the list of non-null
+				 * datums for this partition.
+				 */
+				if (spec->is_default)
+				{
+					default_index = i;
+					i++;
+					continue;
+				}
+
 				foreach(c, spec->listdatums)
 				{
 					Const	   *val = castNode(Const, lfirst(c));
@@ -506,6 +526,22 @@ RelationBuildPartitionDesc(Relation rel)
 					else
 						boundinfo->null_index = -1;
 
+					/* Assign mapped index for the default partition. */
+					if (default_index != -1)
+					{
+						/*
+						 * The default partition accepts any non-specified
+						 * value, hence it should not get a mapped index while
+						 * assigning those for non-null datums.
+						 */
+						Assert(default_index >= 0 &&
+							   mapping[default_index] == -1);
+						mapping[default_index] = next_index++;
+						boundinfo->default_index = mapping[default_index];
+					}
+					else
+						boundinfo->default_index = -1;
+
 					/* All partition must now have a valid mapping */
 					Assert(next_index == nparts);
 					break;
@@ -561,6 +597,9 @@ RelationBuildPartitionDesc(Relation rel)
 						}
 					}
 					boundinfo->indexes[i] = -1;
+
+					/* As of now, we do not support default range partition. */
+					boundinfo->default_index = -1;
 					break;
 				}
 
@@ -610,6 +649,9 @@ partition_bounds_equal(PartitionKey key,
 	if (b1->null_index != b2->null_index)
 		return false;
 
+	if (b1->default_index != b2->default_index)
+		return false;
+
 	for (i = 0; i < b1->ndatums; i++)
 	{
 		int			j;
@@ -672,6 +714,7 @@ check_new_partition_bound(char *relname, Relation parent,
 {
 	PartitionKey key = RelationGetPartitionKey(parent);
 	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+	PartitionBoundInfo boundinfo = partdesc->boundinfo;
 	ParseState *pstate = make_parsestate(NULL);
 	int			with = -1;
 	bool		overlap = false;
@@ -684,13 +727,24 @@ check_new_partition_bound(char *relname, Relation parent,
 
 				if (partdesc->nparts > 0)
 				{
-					PartitionBoundInfo boundinfo = partdesc->boundinfo;
 					ListCell   *cell;
 
 					Assert(boundinfo &&
 						   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
 						   (boundinfo->ndatums > 0 ||
-							partition_bound_accepts_nulls(boundinfo)));
+							partition_bound_accepts_nulls(boundinfo) ||
+							partition_bound_has_default(boundinfo)));
+
+					/*
+					 * Default partition cannot be added if there already
+					 * exists one.
+					 */
+					if (spec->is_default)
+					{
+						overlap = partition_bound_has_default(boundinfo);
+						with = boundinfo->default_index;
+						break;
+					}
 
 					foreach(cell, spec->listdatums)
 					{
@@ -832,12 +886,150 @@ check_new_partition_bound(char *relname, Relation parent,
 	if (overlap)
 	{
 		Assert(with >= 0);
+
+		if (spec->is_default)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+					 errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+							relname, get_rel_name(partdesc->oids[with])),
+					 parser_errposition(pstate, spec->location)));
+
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
 				 errmsg("partition \"%s\" would overlap partition \"%s\"",
 						relname, get_rel_name(partdesc->oids[with])),
 				 parser_errposition(pstate, spec->location)));
 	}
+
+	/*
+	 * If the default partition exists, its partition constraints will change
+	 * after the addition of this new partition such that it won't allow any
+	 * row that qualifies for this new partition. So, check if the existing
+	 * data in the default partition satisfies this *would be* default
+	 * partition constraint.
+	 * In case the new partition bound being checked itself is a DEFAULT
+	 * bound, this check shouldn't be triggered as there won't already exists
+	 * the default partition in such a case.
+	 */
+	if (boundinfo && partition_bound_has_default(boundinfo))
+		check_default_allows_bound(parent, spec);
+}
+
+/*
+ * check_default_allows_bound
+ *
+ * This function checks if there exists a row in the default partition that
+ * fits in the new partition and throws an error if it finds one.
+ */
+static void
+check_default_allows_bound(Relation parent, PartitionBoundSpec *new_spec)
+{
+	Relation	default_rel;
+	int			default_index;
+	List	   *new_part_constraints = NIL;
+	List	   *all_parts;
+	ListCell   *lc;
+
+	/* Currently default partition is supported only for LIST partition. */
+	Assert(new_spec->strategy == PARTITION_STRATEGY_LIST);
+
+	/* If there exists a default partition, then boundinfo cannot be NULL */
+	Assert(RelationGetPartitionDesc(parent)->boundinfo != NULL);
+
+	new_part_constraints = get_qual_for_list(parent, new_spec);
+	new_part_constraints = (List *) eval_const_expressions(NULL,
+											  (Node *) new_part_constraints);
+	new_part_constraints =
+		(List *) canonicalize_qual((Expr *) new_part_constraints);
+	new_part_constraints = list_make1(make_ands_explicit(new_part_constraints));
+
+	/* Generate the constraint and default execution states. */
+	default_index = RelationGetPartitionDesc(parent)->boundinfo->default_index;
+	default_rel =
+		heap_open(RelationGetPartitionDesc(parent)->oids[default_index],
+				  AccessExclusiveLock);
+
+	if (default_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+		all_parts = find_all_inheritors(RelationGetRelid(default_rel),
+										AccessExclusiveLock, NULL);
+	else
+		all_parts = list_make1_oid(RelationGetRelid(default_rel));
+
+	foreach(lc, all_parts)
+	{
+		Oid			part_relid = lfirst_oid(lc);
+		Relation	part_rel;
+		Expr	   *constr;
+		Expr	   *partition_constraint;
+		EState	   *estate;
+		HeapTuple	tuple;
+		ExprState  *partqualstate = NULL;
+		Snapshot	snapshot;
+		TupleDesc	tupdesc;
+		ExprContext *econtext;
+		HeapScanDesc scan;
+		MemoryContext oldCxt;
+		TupleTableSlot *tupslot;
+
+		/* Lock already taken above. */
+		if (part_relid != RelationGetRelid(default_rel))
+			part_rel = heap_open(part_relid, NoLock);
+		else
+			part_rel = default_rel;
+
+		/*
+		 * Skip if it's a partitioned table. Only RELKIND_RELATION relations
+		 * (ie, leaf partitions) need to be scanned.
+		 */
+		if (part_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ||
+			part_rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+		{
+			heap_close(part_rel, NoLock);
+			continue;
+		}
+
+		tupdesc = CreateTupleDescCopy(RelationGetDescr(part_rel));
+		constr = linitial(new_part_constraints);
+		partition_constraint = (Expr *) map_partition_varattnos((List *) constr,
+														1, part_rel, parent);
+		estate = CreateExecutorState();
+
+		/* Build expression execution states for partition check quals */
+		partqualstate = ExecPrepareExpr(partition_constraint, estate);
+
+		econtext = GetPerTupleExprContext(estate);
+		snapshot = RegisterSnapshot(GetLatestSnapshot());
+		scan = heap_beginscan(part_rel, snapshot, 0, NULL);
+		tupslot = MakeSingleTupleTableSlot(tupdesc);
+
+		/*
+		 * Switch to per-tuple memory context and reset it for each tuple
+		 * produced, so we don't leak memory.
+		 */
+		oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
+
+		while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
+		{
+			ExecStoreTuple(tuple, tupslot, InvalidBuffer, false);
+			econtext->ecxt_scantuple = tupslot;
+
+			if (partqualstate && ExecCheck(partqualstate, econtext))
+				ereport(ERROR,
+						(errcode(ERRCODE_CHECK_VIOLATION),
+						 errmsg("new partition constraint for default partition \"%s\" would be violated by some row",
+							   RelationGetRelationName(default_rel))));
+
+			ResetExprContext(econtext);
+			CHECK_FOR_INTERRUPTS();
+		}
+
+		MemoryContextSwitchTo(oldCxt);
+		heap_endscan(scan);
+		UnregisterSnapshot(snapshot);
+		ExecDropSingleTupleTableSlot(tupslot);
+		FreeExecutorState(estate);
+		heap_close(part_rel, NoLock);
+	}
 }
 
 /*
@@ -904,7 +1096,7 @@ get_qual_from_partbound(Relation rel, Relation parent,
 	{
 		case PARTITION_STRATEGY_LIST:
 			Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-			my_qual = get_qual_for_list(key, spec);
+			my_qual = get_qual_for_list(parent, spec);
 			break;
 
 		case PARTITION_STRATEGY_RANGE:
@@ -1234,7 +1426,7 @@ get_partition_operator(PartitionKey key, int col, StrategyNumber strategy,
  */
 static Expr *
 make_partition_op_expr(PartitionKey key, int keynum,
-					   uint16 strategy, Expr *arg1, Expr *arg2)
+					   uint16 strategy, Expr *arg1, Expr *arg2, bool is_default)
 {
 	Oid			operoid;
 	bool		need_relabel = false;
@@ -1264,11 +1456,21 @@ make_partition_op_expr(PartitionKey key, int keynum,
 			{
 				ScalarArrayOpExpr *saopexpr;
 
-				/* Build leftop = ANY (rightop) */
+				if (is_default &&
+					((operoid = get_negator(operoid)) == InvalidOid))
+					ereport(ERROR, (errcode(ERRCODE_RESTRICT_VIOLATION),
+									errmsg("DEFAULT partition cannot be used without negator of operator  %s",
+										   get_opname(operoid))));
+
+				/*
+				 * Build expression:
+				 * "leftop = ANY (rightop)" for a non-default partition OR
+				 * "leftop <> ALL (rightop)" for the default partition.
+				 */
 				saopexpr = makeNode(ScalarArrayOpExpr);
 				saopexpr->opno = operoid;
 				saopexpr->opfuncid = get_opcode(operoid);
-				saopexpr->useOr = true;
+				saopexpr->useOr = !is_default;
 				saopexpr->inputcollid = key->partcollation[keynum];
 				saopexpr->args = list_make2(arg1, arg2);
 				saopexpr->location = -1;
@@ -1301,8 +1503,9 @@ make_partition_op_expr(PartitionKey key, int keynum,
  * constraint, given the partition key and bound structures.
  */
 static List *
-get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec)
+get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 {
+	PartitionKey key = RelationGetPartitionKey(parent);
 	List	   *result;
 	Expr	   *keyCol;
 	ArrayExpr  *arr;
@@ -1323,15 +1526,60 @@ get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec)
 	else
 		keyCol = (Expr *) copyObject(linitial(key->partexprs));
 
-	/* Create list of Consts for the allowed values, excluding any nulls */
-	foreach(cell, spec->listdatums)
+	/*
+	 * For default list partition, collect datums for all the partitions. The
+	 * default partition constraint should check that the partition key is
+	 * equal to none of those.
+	 */
+	if (spec->is_default)
 	{
-		Const	   *val = castNode(Const, lfirst(cell));
+		int         i;
+		int         ndatums = 0;
+		MemoryContext oldcxt;
+		PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+		PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
-		if (val->constisnull)
+		if (boundinfo)
+			ndatums = boundinfo->ndatums;
+
+		oldcxt = MemoryContextSwitchTo(CacheMemoryContext);
+
+		for (i = 0; i < ndatums; i++)
+		{
+			Const      *val;
+
+			/* Construct const from datum */
+			val = makeConst(key->parttypid[0],
+							key->parttypmod[0],
+							key->parttypcoll[0],
+							key->parttyplen[0],
+							*boundinfo->datums[i],
+							false,      /* isnull */
+							true /* byval */ );
+
+			arrelems = lappend(arrelems, val);
+		}
+
+		if (boundinfo && partition_bound_accepts_nulls(boundinfo))
 			list_has_null = true;
-		else
-			arrelems = lappend(arrelems, copyObject(val));
+
+		MemoryContextSwitchTo(oldcxt);
+	}
+	else
+	{
+		/*
+		 * Create list of Consts for the allowed values, excluding any
+		 * nulls.
+		 */
+		foreach(cell, spec->listdatums)
+		{
+			Const	   *val = castNode(Const, lfirst(cell));
+
+			if (val->constisnull)
+				list_has_null = true;
+			else
+				arrelems = lappend(arrelems, copyObject(val));
+		}
 	}
 
 	/* Construct an ArrayExpr for the non-null partition values */
@@ -1345,11 +1593,22 @@ get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec)
 	arr->multidims = false;
 	arr->location = -1;
 
-	/* Generate the main expression, i.e., keyCol = ANY (arr) */
+	/*
+	 * Generate the main expression as below:
+	 * in case of the default partition: keyCol <> ALL (arr)
+	 * or for a non-default partition: keyCol = ANY (arr)
+	 */
 	opexpr = make_partition_op_expr(key, 0, BTEqualStrategyNumber,
-									keyCol, (Expr *) arr);
+									keyCol, (Expr *) arr, spec->is_default);
 
-	if (!list_has_null)
+	/*
+	 * If this isn't a default partition and does not accept a null or if this
+	 * is a default partition and there exists another partition accepting null;
+	 * in any of these two cases this partition cannot accept null, and we need
+	 * to build a IS_NOT_NULL constraint for it.
+	 */
+	if ((!list_has_null && !spec->is_default) ||
+		(list_has_null && spec->is_default))
 	{
 		/*
 		 * Gin up a "col IS NOT NULL" test that will be AND'd with the main
@@ -1592,7 +1851,7 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
 		oldcxt = MemoryContextSwitchTo(estate->es_query_cxt);
 		test_expr = make_partition_op_expr(key, i, BTEqualStrategyNumber,
 										   (Expr *) lower_val,
-										   (Expr *) upper_val);
+										   (Expr *) upper_val, false);
 		fix_opfuncids((Node *) test_expr);
 		test_exprstate = ExecInitExpr(test_expr, NULL);
 		test_result = ExecEvalExprSwitchContext(test_exprstate,
@@ -1616,7 +1875,8 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
 		/* Equal, so generate keyCol = lower_val expression */
 		result = lappend(result,
 						 make_partition_op_expr(key, i, BTEqualStrategyNumber,
-												keyCol, (Expr *) lower_val));
+												keyCol, (Expr *) lower_val,
+												false));
 
 		i++;
 	}
@@ -1677,7 +1937,8 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
 											make_partition_op_expr(key, j,
 																   strategy,
 																   keyCol,
-														(Expr *) lower_val));
+														(Expr *) lower_val,
+																   false));
 			}
 
 			if (need_next_upper_arm && upper_val)
@@ -1699,7 +1960,8 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
 											make_partition_op_expr(key, j,
 																   strategy,
 																   keyCol,
-														(Expr *) upper_val));
+														(Expr *) upper_val,
+																   false));
 			}
 
 			/*
@@ -1989,8 +2251,9 @@ get_partition_for_tuple(PartitionDispatch *pd,
 		}
 
 		/*
-		 * A null partition key is only acceptable if null-accepting list
-		 * partition exists.
+		 * Handle NULL partition key here if there's a null-accepting list
+		 * partition, else later it will be routed to the default partition if
+		 * one exists.
 		 */
 		cur_index = -1;
 		if (isnull[0] && partition_bound_accepts_nulls(partdesc->boundinfo))
@@ -2028,10 +2291,16 @@ get_partition_for_tuple(PartitionDispatch *pd,
 		}
 
 		/*
-		 * cur_index < 0 means we failed to find a partition of this parent.
-		 * cur_index >= 0 means we either found the leaf partition, or the
-		 * next parent to find a partition of.
+		 * cur_index < 0 means we could not find a non-default partition of this
+		 * parent. cur_index >= 0 means we either found the leaf partition, or
+		 * the next parent to find a partition of.
+		 *
+		 * If we couldn't find a non-default partition check if the default
+		 * partition exists, if it does, get its index.
 		 */
+		if (cur_index < 0 && (partition_bound_has_default(partdesc->boundinfo)))
+			cur_index = partdesc->boundinfo->default_index;
+
 		if (cur_index < 0)
 		{
 			result = -1;
@@ -2313,3 +2582,20 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+/*
+ * RelationGetDefaultPartitionOid
+ *
+ * Given the parent relation checks if it has default partition, and if it
+ * does exist returns its oid, otherwise returns InvalidOid.
+ */
+Oid
+RelationGetDefaultPartitionOid(Relation parent)
+{
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+
+	if (partdesc->boundinfo && partition_bound_has_default(partdesc->boundinfo))
+		return partdesc->oids[partdesc->boundinfo->default_index];
+
+	return InvalidOid;
+}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index b61fda9..bdce229 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -758,7 +758,8 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 	{
 		PartitionBoundSpec *bound;
 		ParseState *pstate;
-		Oid			parentId = linitial_oid(inheritOids);
+		Oid			parentId = linitial_oid(inheritOids),
+					defaultPartOid;
 		Relation	parent;
 
 		/* Already have strong enough lock on the parent */
@@ -774,6 +775,20 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 					 errmsg("\"%s\" is not partitioned",
 							RelationGetRelationName(parent))));
 
+		/*
+		 * We need to take an exclusive lock on the default partition if it
+		 * exists, because its constraints are dependent upon other partition
+		 * bounds. It is possible that other backend might be about to execute
+		 * a query on the default partition table and the query relies on
+		 * previously cached default partition constraints; those constraints
+		 * won't stand correct after addition(or even removal) of a partition.
+		 * We must therefore take an exclusive lock to prevent all queries on
+		 * the default partition table from proceeding until we commit.
+		 */
+		defaultPartOid = RelationGetDefaultPartitionOid(parent);
+		if (OidIsValid(defaultPartOid))
+			LockRelationOid(defaultPartOid, AccessExclusiveLock);
+
 		/* Tranform the bound values */
 		pstate = make_parsestate(NULL);
 		pstate->p_sourcetext = queryString;
@@ -13798,6 +13813,7 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 				new_null[Natts_pg_class],
 				new_repl[Natts_pg_class];
 	ObjectAddress address;
+	Oid			defaultPartOid;
 
 	partRel = heap_openrv(name, AccessShareLock);
 
@@ -13830,6 +13846,14 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
 	heap_close(classRel, RowExclusiveLock);
 
 	/*
+	 * Invalidate default partition's relcache, refer comment for invalidating
+	 * default partition relcache in StorePartitionBound().
+	 */
+	defaultPartOid = RelationGetDefaultPartitionOid(rel);
+	if (OidIsValid(defaultPartOid))
+		CacheInvalidateRelcacheByRelid(defaultPartOid);
+
+	/*
 	 * Invalidate the parent's relcache so that the partition is no longer
 	 * included in its partition descriptor.
 	 */
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 36bf1dc..03d2df9 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -4444,6 +4444,7 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
 	PartitionBoundSpec *newnode = makeNode(PartitionBoundSpec);
 
 	COPY_SCALAR_FIELD(strategy);
+	COPY_SCALAR_FIELD(is_default);
 	COPY_NODE_FIELD(listdatums);
 	COPY_NODE_FIELD(lowerdatums);
 	COPY_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 5bcf031..01fd43d 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -2838,6 +2838,7 @@ static bool
 _equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *b)
 {
 	COMPARE_SCALAR_FIELD(strategy);
+	COMPARE_SCALAR_FIELD(is_default);
 	COMPARE_NODE_FIELD(listdatums);
 	COMPARE_NODE_FIELD(lowerdatums);
 	COMPARE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index c348bdc..ae66d3d 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -3543,6 +3543,7 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
 	WRITE_NODE_TYPE("PARTITIONBOUNDSPEC");
 
 	WRITE_CHAR_FIELD(strategy);
+	WRITE_BOOL_FIELD(is_default);
 	WRITE_NODE_FIELD(listdatums);
 	WRITE_NODE_FIELD(lowerdatums);
 	WRITE_NODE_FIELD(upperdatums);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 81ddfc3..290a0bb 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2373,6 +2373,7 @@ _readPartitionBoundSpec(void)
 	READ_LOCALS(PartitionBoundSpec);
 
 	READ_CHAR_FIELD(strategy);
+	READ_BOOL_FIELD(is_default);
 	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 7e03624..a73e506 100644
--- a/src/backend/parser/gram.y
+++ b/src/backend/parser/gram.y
@@ -2657,6 +2657,7 @@ ForValues:
 					PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
 
 					n->strategy = PARTITION_STRATEGY_LIST;
+					n->is_default = false;
 					n->listdatums = $5;
 					n->location = @3;
 
@@ -2669,12 +2670,27 @@ ForValues:
 					PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
 
 					n->strategy = PARTITION_STRATEGY_RANGE;
+					n->is_default = false;
 					n->lowerdatums = $5;
 					n->upperdatums = $9;
 					n->location = @3;
 
 					$$ = n;
 				}
+
+			/*
+			 * A default partition, that can be partition of either
+			 * LIST or RANGE partitioned table.
+			 */
+			| DEFAULT
+				{
+					PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+					n->is_default = true;
+					n->location = @1;
+
+					$$ = n;
+				}
 		;
 
 partbound_datum:
diff --git a/src/backend/parser/parse_utilcmd.c b/src/backend/parser/parse_utilcmd.c
index 9134fb9..dd9d736 100644
--- a/src/backend/parser/parse_utilcmd.c
+++ b/src/backend/parser/parse_utilcmd.c
@@ -33,6 +33,7 @@
 #include "catalog/heap.h"
 #include "catalog/index.h"
 #include "catalog/namespace.h"
+#include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "catalog/pg_collation.h"
 #include "catalog/pg_constraint.h"
@@ -60,6 +61,7 @@
 #include "parser/parse_utilcmd.h"
 #include "parser/parser.h"
 #include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
 #include "utils/acl.h"
 #include "utils/builtins.h"
 #include "utils/lsyscache.h"
@@ -3250,6 +3252,7 @@ static void
 transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd)
 {
 	Relation	parentRel = cxt->rel;
+	Oid			defaultPartOid;
 
 	/* the table must be partitioned */
 	if (parentRel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
@@ -3258,6 +3261,14 @@ transformPartitionCmd(CreateStmtContext *cxt, PartitionCmd *cmd)
 				 errmsg("\"%s\" is not partitioned",
 						RelationGetRelationName(parentRel))));
 
+	/*
+	 * Need to take a lock on the default partition, refer comment for locking
+	 * the default partition in DefineRelation().
+	 */
+	defaultPartOid = RelationGetDefaultPartitionOid(parentRel);
+	if (OidIsValid(defaultPartOid))
+		LockRelationOid(defaultPartOid, AccessExclusiveLock);
+
 	/* transform the partition bound, if any */
 	Assert(RelationGetPartitionKey(parentRel) != NULL);
 	if (cmd->bound != NULL)
@@ -3283,6 +3294,23 @@ transformPartitionBound(ParseState *pstate, Relation parent,
 	/* Avoid scribbling on input */
 	result_spec = copyObject(spec);
 
+	if (spec->is_default)
+	{
+		if (strategy != PARTITION_STRATEGY_LIST)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_TABLE_DEFINITION),
+					 errmsg("default partition is supported only for a list partitioned table")));
+
+		/*
+		 * In case of the default partition, parser had no way to identify the
+		 * partition strategy. Assign the parent strategy to the default
+		 * partition bound spec.
+		 */
+		result_spec->strategy = strategy;
+
+		return result_spec;
+	}
+
 	if (strategy == PARTITION_STRATEGY_LIST)
 	{
 		ListCell   *cell;
diff --git a/src/backend/utils/adt/ruleutils.c b/src/backend/utils/adt/ruleutils.c
index 6a0d273..0c1e72a 100644
--- a/src/backend/utils/adt/ruleutils.c
+++ b/src/backend/utils/adt/ruleutils.c
@@ -8644,10 +8644,18 @@ get_rule_expr(Node *node, deparse_context *context,
 		case T_PartitionBoundSpec:
 			{
 				PartitionBoundSpec *spec = (PartitionBoundSpec *) node;
+				char		strategy = spec->strategy;
 				ListCell   *cell;
 				char	   *sep;
 
-				switch (spec->strategy)
+				if (spec->is_default)
+				{
+					Assert(strategy == PARTITION_STRATEGY_LIST);
+					appendStringInfoString(buf, "DEFAULT");
+					break;
+				}
+
+				switch (strategy)
 				{
 					case PARTITION_STRATEGY_LIST:
 						Assert(spec->listdatums != NIL);
@@ -8713,7 +8721,7 @@ get_rule_expr(Node *node, deparse_context *context,
 
 					default:
 						elog(ERROR, "unrecognized partition strategy: %d",
-							 (int) spec->strategy);
+							 (int) strategy);
 						break;
 				}
 			}
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index d4b6976..00fcbd3 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -2058,7 +2058,7 @@ psql_completion(const char *text, int start, int end)
 		COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_tables, "");
 	/* Limited completion support for partition bound specification */
 	else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
-		COMPLETE_WITH_CONST("FOR VALUES");
+		COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
 	else if (TailMatches2("FOR", "VALUES"))
 		COMPLETE_WITH_LIST2("FROM (", "IN (");
 
@@ -2497,7 +2497,7 @@ psql_completion(const char *text, int start, int end)
 		COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_partitioned_tables, "");
 	/* Limited completion support for partition bound specification */
 	else if (TailMatches3("PARTITION", "OF", MatchAny))
-		COMPLETE_WITH_CONST("FOR VALUES");
+		COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
 
 /* CREATE TABLESPACE */
 	else if (Matches3("CREATE", "TABLESPACE", MatchAny))
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 0a1e468..e4e079b 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -98,4 +98,5 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 						EState *estate,
 						PartitionDispatchData **failed_at,
 						TupleTableSlot **failed_slot);
+extern Oid RelationGetDefaultPartitionOid(Relation parent);
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 8720e71..e717778 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -798,6 +798,7 @@ typedef struct PartitionBoundSpec
 	NodeTag		type;
 
 	char		strategy;		/* see PARTITION_STRATEGY codes above */
+	bool		is_default;		/* is it a default partition bound? */
 
 	/* Partitioning info for LIST strategy: */
 	List	   *listdatums;		/* List of Consts (or A_Consts in raw tree) */
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index 8aadbb8..5c59028 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3273,6 +3273,11 @@ 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"
+-- attaching default partition overlaps if the default partition already exists
+CREATE TABLE def_part PARTITION OF list_parted DEFAULT;
+CREATE TABLE fail_def_part (LIKE part_1 INCLUDING CONSTRAINTS);
+ALTER TABLE list_parted ATTACH PARTITION fail_def_part DEFAULT;
+ERROR:  partition "fail_def_part" conflicts with existing default partition "def_part"
 -- check validation when attaching list partitions
 CREATE TABLE list_parted2 (
 	a int,
@@ -3286,6 +3291,15 @@ ERROR:  partition constraint is violated by some row
 -- should be ok after deleting the bad row
 DELETE FROM part_2;
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
+-- check partition cannot be attached if default has some row for its values
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
+INSERT INTO list_parted2_def VALUES (11, 'z');
+CREATE TABLE part_3 (LIKE list_parted2);
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
+ERROR:  new partition constraint for default partition "list_parted2_def" would be violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM list_parted2_def WHERE a = 11;
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
 -- adding constraints that describe the desired partition constraint
 -- (or more restrictive) will help skip the validation scan
 CREATE TABLE part_3_4 (
@@ -3341,6 +3355,18 @@ DELETE FROM part_5_a WHERE a NOT IN (3);
 ALTER TABLE part_5 ADD CONSTRAINT check_a CHECK (a IN (5)), ALTER a SET NOT NULL;
 ALTER TABLE list_parted2 ATTACH PARTITION part_5 FOR VALUES IN (5);
 INFO:  partition constraint for table "part_5" is implied by existing constraints
+-- check that leaf partitions of default partition are scanned when
+-- attaching a partitioned table.
+ALTER TABLE part_5 DROP CONSTRAINT check_a;
+CREATE TABLE part5_def PARTITION OF part_5 DEFAULT PARTITION BY LIST(a);
+CREATE TABLE part5_def_p1 PARTITION OF part5_def FOR VALUES IN (5);
+INSERT INTO part5_def_p1 VALUES (5, 'y');
+CREATE TABLE part5_p1 (LIKE part_5);
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
+ERROR:  new partition constraint for default partition "part5_def" would be violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM part5_def_p1 WHERE b = 'y';
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
 -- check that the table being attached is not already a partition
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 ERROR:  "part_2" is already a partition
diff --git a/src/test/regress/expected/create_table.out b/src/test/regress/expected/create_table.out
index 5136506..81f45d0 100644
--- a/src/test/regress/expected/create_table.out
+++ b/src/test/regress/expected/create_table.out
@@ -449,6 +449,7 @@ CREATE TABLE list_parted (
 CREATE TABLE part_1 PARTITION OF list_parted FOR VALUES IN ('1');
 CREATE TABLE part_2 PARTITION OF list_parted FOR VALUES IN (2);
 CREATE TABLE part_null PARTITION OF list_parted FOR VALUES IN (null);
+CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
 CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN (int '1');
 ERROR:  syntax error at or near "int"
 LINE 1: ... fail_part PARTITION OF list_parted FOR VALUES IN (int '1');
@@ -457,6 +458,8 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
 ERROR:  syntax error at or near "::"
 LINE 1: ...fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
                                                                 ^
+CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
+ERROR:  partition "fail_default_part" conflicts with existing default partition "part_default"
 -- syntax does not allow empty list of values for list partitions
 CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
 ERROR:  syntax error at or near ")"
@@ -514,6 +517,9 @@ 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
+-- range partition cannot have default partition
+CREATE TABLE fail_part PARTITION OF range_parted DEFAULT;
+ERROR:  default partition is supported only for a list partitioned table
 -- cannot specify finite values after UNBOUNDED has been specified
 CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a, b, c);
 CREATE TABLE fail_part PARTITION OF range_parted_multicol FOR VALUES FROM (1, UNBOUNDED, 1) TO (UNBOUNDED, 1, 1);
@@ -565,10 +571,15 @@ CREATE TABLE list_parted2 (
 ) PARTITION BY LIST (a);
 CREATE TABLE part_null_z PARTITION OF list_parted2 FOR VALUES IN (null, 'z');
 CREATE TABLE part_ab PARTITION OF list_parted2 FOR VALUES IN ('a', 'b');
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN (null);
 ERROR:  partition "fail_part" would overlap partition "part_null_z"
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('b', 'c');
 ERROR:  partition "fail_part" would overlap partition "part_ab"
+-- check default partition overlap
+INSERT INTO list_parted2 VALUES('X');
+CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('W', 'X', 'Y');
+ERROR:  new partition constraint for default partition "list_parted2_def" would be violated by some row
 CREATE TABLE range_parted2 (
 	a int
 ) PARTITION BY RANGE (a);
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index 8b0752a..f9eef3f 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -202,6 +202,7 @@ create table list_parted (
 create table part_aa_bb partition of list_parted FOR VALUES IN ('aa', 'bb');
 create table part_cc_dd partition of list_parted FOR VALUES IN ('cc', 'dd');
 create table part_null partition of list_parted FOR VALUES IN (null);
+create table part_default partition of list_parted default;
 -- fail
 insert into part_aa_bb values ('cc', 1);
 ERROR:  new row for relation "part_aa_bb" violates partition constraint
@@ -212,24 +213,45 @@ DETAIL:  Failing row contains (AAa, 1).
 insert into part_aa_bb values (null);
 ERROR:  new row for relation "part_aa_bb" violates partition constraint
 DETAIL:  Failing row contains (null, null).
+insert into part_default values ('aa', 2);
+ERROR:  new row for relation "part_default" violates partition constraint
+DETAIL:  Failing row contains (aa, 2).
+insert into part_default values (null, 2);
+ERROR:  new row for relation "part_default" violates partition constraint
+DETAIL:  Failing row contains (null, 2).
 -- ok
 insert into part_cc_dd values ('cC', 1);
 insert into part_null values (null, 0);
+insert into part_default values ('Zz', 2);
 -- check in case of multi-level partitioned table
 create table part_ee_ff partition of list_parted for values in ('ee', 'ff') partition by range (b);
 create table part_ee_ff1 partition of part_ee_ff for values from (1) to (10);
 create table part_ee_ff2 partition of part_ee_ff for values from (10) to (20);
+-- check in case of multi-level default partitioned table
+drop table part_default;
+create table part_default partition of list_parted default partition by range(b);
+create table part_default_p1 partition of part_default for values from (20) to (30);
+create table part_default_p2 partition of part_default for values from (30) to (40);
 -- fail
 insert into part_ee_ff1 values ('EE', 11);
 ERROR:  new row for relation "part_ee_ff1" violates partition constraint
 DETAIL:  Failing row contains (EE, 11).
+insert into part_default_p2 values ('gg', 43);
+ERROR:  new row for relation "part_default_p2" violates partition constraint
+DETAIL:  Failing row contains (gg, 43).
 -- fail (even the parent's, ie, part_ee_ff's partition constraint applies)
 insert into part_ee_ff1 values ('cc', 1);
 ERROR:  new row for relation "part_ee_ff1" violates partition constraint
 DETAIL:  Failing row contains (cc, 1).
+insert into part_default values ('gg', 43);
+ERROR:  no partition of relation "part_default" found for row
+DETAIL:  Partition key of the failing row contains (b) = (43).
 -- ok
 insert into part_ee_ff1 values ('ff', 1);
 insert into part_ee_ff2 values ('ff', 11);
+insert into part_default_p1 values ('cd', 25);
+insert into part_default_p2 values ('de', 35);
+insert into list_parted values ('ab', 21);
 -- Check tuple routing for partitioned tables
 -- fail
 insert into range_parted values ('a', 0);
@@ -274,17 +296,20 @@ DETAIL:  Partition key of the failing row contains (b) = (0).
 insert into list_parted values ('EE', 1);
 insert into part_ee_ff values ('EE', 10);
 select tableoid::regclass, * from list_parted;
-  tableoid   | a  | b  
--------------+----+----
- part_aa_bb  | aA |   
- part_cc_dd  | cC |  1
- part_null   |    |  0
- part_null   |    |  1
- part_ee_ff1 | ff |  1
- part_ee_ff1 | EE |  1
- part_ee_ff2 | ff | 11
- part_ee_ff2 | EE | 10
-(8 rows)
+    tableoid     | a  | b  
+-----------------+----+----
+ part_aa_bb      | aA |   
+ part_cc_dd      | cC |  1
+ part_null       |    |  0
+ part_null       |    |  1
+ part_ee_ff1     | ff |  1
+ part_ee_ff1     | EE |  1
+ part_ee_ff2     | ff | 11
+ part_ee_ff2     | EE | 10
+ part_default_p1 | cd | 25
+ part_default_p1 | ab | 21
+ part_default_p2 | de | 35
+(11 rows)
 
 -- some more tests to exercise tuple-routing with multi-level partitioning
 create table part_gg partition of list_parted for values in ('gg') partition by range (b);
diff --git a/src/test/regress/expected/update.out b/src/test/regress/expected/update.out
index 9366f04..9912ef2 100644
--- a/src/test/regress/expected/update.out
+++ b/src/test/regress/expected/update.out
@@ -218,5 +218,20 @@ 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 list_parted (
+	a text,
+	b int
+) partition by list (a);
+create table list_part1  partition of list_parted for values in ('a', 'b');
+create table list_default partition of list_parted default;
+insert into list_part1 values ('a', 1);
+insert into list_default values ('d', 10);
+-- fail
+update list_default set a = 'a' where a = 'd';
+ERROR:  new row for relation "list_default" violates partition constraint
+DETAIL:  Failing row contains (a, 10).
+-- ok
+update list_default set a = 'x' where a = 'd';
 -- cleanup
 drop table range_parted;
+drop table list_parted;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index c41b487..36c56aa 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2095,6 +2095,10 @@ 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);
+-- attaching default partition overlaps if the default partition already exists
+CREATE TABLE def_part PARTITION OF list_parted DEFAULT;
+CREATE TABLE fail_def_part (LIKE part_1 INCLUDING CONSTRAINTS);
+ALTER TABLE list_parted ATTACH PARTITION fail_def_part DEFAULT;
 
 -- check validation when attaching list partitions
 CREATE TABLE list_parted2 (
@@ -2111,6 +2115,15 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 DELETE FROM part_2;
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 
+-- check partition cannot be attached if default has some row for its values
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
+INSERT INTO list_parted2_def VALUES (11, 'z');
+CREATE TABLE part_3 (LIKE list_parted2);
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
+-- should be ok after deleting the bad row
+DELETE FROM list_parted2_def WHERE a = 11;
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
+
 -- adding constraints that describe the desired partition constraint
 -- (or more restrictive) will help skip the validation scan
 CREATE TABLE part_3_4 (
@@ -2172,6 +2185,17 @@ DELETE FROM part_5_a WHERE a NOT IN (3);
 ALTER TABLE part_5 ADD CONSTRAINT check_a CHECK (a IN (5)), ALTER a SET NOT NULL;
 ALTER TABLE list_parted2 ATTACH PARTITION part_5 FOR VALUES IN (5);
 
+-- check that leaf partitions of default partition are scanned when
+-- attaching a partitioned table.
+ALTER TABLE part_5 DROP CONSTRAINT check_a;
+CREATE TABLE part5_def PARTITION OF part_5 DEFAULT PARTITION BY LIST(a);
+CREATE TABLE part5_def_p1 PARTITION OF part5_def FOR VALUES IN (5);
+INSERT INTO part5_def_p1 VALUES (5, 'y');
+CREATE TABLE part5_p1 (LIKE part_5);
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
+-- should be ok after deleting the bad row
+DELETE FROM part5_def_p1 WHERE b = 'y';
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
 
 -- check that the table being attached is not already a partition
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
diff --git a/src/test/regress/sql/create_table.sql b/src/test/regress/sql/create_table.sql
index cb7aa5b..f44c0e0 100644
--- a/src/test/regress/sql/create_table.sql
+++ b/src/test/regress/sql/create_table.sql
@@ -439,8 +439,10 @@ CREATE TABLE list_parted (
 CREATE TABLE part_1 PARTITION OF list_parted FOR VALUES IN ('1');
 CREATE TABLE part_2 PARTITION OF list_parted FOR VALUES IN (2);
 CREATE TABLE part_null PARTITION OF list_parted FOR VALUES IN (null);
+CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
 CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN (int '1');
 CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ('1'::int);
+CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
 
 -- syntax does not allow empty list of values for list partitions
 CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
@@ -484,6 +486,8 @@ 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);
+-- range partition cannot have default partition
+CREATE TABLE fail_part PARTITION OF range_parted DEFAULT;
 
 -- cannot specify finite values after UNBOUNDED has been specified
 CREATE TABLE range_parted_multicol (a int, b int, c int) PARTITION BY RANGE (a, b, c);
@@ -529,9 +533,13 @@ CREATE TABLE list_parted2 (
 ) PARTITION BY LIST (a);
 CREATE TABLE part_null_z PARTITION OF list_parted2 FOR VALUES IN (null, 'z');
 CREATE TABLE part_ab PARTITION OF list_parted2 FOR VALUES IN ('a', 'b');
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
 
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN (null);
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('b', 'c');
+-- check default partition overlap
+INSERT INTO list_parted2 VALUES('X');
+CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('W', 'X', 'Y');
 
 CREATE TABLE range_parted2 (
 	a int
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index db8967b..c120713 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -118,27 +118,41 @@ create table list_parted (
 create table part_aa_bb partition of list_parted FOR VALUES IN ('aa', 'bb');
 create table part_cc_dd partition of list_parted FOR VALUES IN ('cc', 'dd');
 create table part_null partition of list_parted FOR VALUES IN (null);
+create table part_default partition of list_parted default;
 
 -- fail
 insert into part_aa_bb values ('cc', 1);
 insert into part_aa_bb values ('AAa', 1);
 insert into part_aa_bb values (null);
+insert into part_default values ('aa', 2);
+insert into part_default values (null, 2);
 -- ok
 insert into part_cc_dd values ('cC', 1);
 insert into part_null values (null, 0);
+insert into part_default values ('Zz', 2);
 
 -- check in case of multi-level partitioned table
 create table part_ee_ff partition of list_parted for values in ('ee', 'ff') partition by range (b);
 create table part_ee_ff1 partition of part_ee_ff for values from (1) to (10);
 create table part_ee_ff2 partition of part_ee_ff for values from (10) to (20);
+-- check in case of multi-level default partitioned table
+drop table part_default;
+create table part_default partition of list_parted default partition by range(b);
+create table part_default_p1 partition of part_default for values from (20) to (30);
+create table part_default_p2 partition of part_default for values from (30) to (40);
 
 -- fail
 insert into part_ee_ff1 values ('EE', 11);
+insert into part_default_p2 values ('gg', 43);
 -- fail (even the parent's, ie, part_ee_ff's partition constraint applies)
 insert into part_ee_ff1 values ('cc', 1);
+insert into part_default values ('gg', 43);
 -- ok
 insert into part_ee_ff1 values ('ff', 1);
 insert into part_ee_ff2 values ('ff', 11);
+insert into part_default_p1 values ('cd', 25);
+insert into part_default_p2 values ('de', 35);
+insert into list_parted values ('ab', 21);
 
 -- Check tuple routing for partitioned tables
 
diff --git a/src/test/regress/sql/update.sql b/src/test/regress/sql/update.sql
index 6637119..44fb0dc 100644
--- a/src/test/regress/sql/update.sql
+++ b/src/test/regress/sql/update.sql
@@ -125,5 +125,20 @@ update range_parted set b = b - 1 where b = 10;
 -- ok
 update range_parted set b = b + 1 where b = 10;
 
+create table list_parted (
+	a text,
+	b int
+) partition by list (a);
+create table list_part1  partition of list_parted for values in ('a', 'b');
+create table list_default partition of list_parted default;
+insert into list_part1 values ('a', 1);
+insert into list_default values ('d', 10);
+
+-- fail
+update list_default set a = 'a' where a = 'd';
+-- ok
+update list_default set a = 'x' where a = 'd';
+
 -- cleanup
 drop table range_parted;
+drop table list_parted;
